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
deleted file mode 100644
index 4fdae9d92..000000000
--- a/app/org/apache/spark/deploy/history/SparkDataCollection.scala
+++ /dev/null
@@ -1,317 +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.deploy.history
-
-
-import com.linkedin.drelephant.analysis.ApplicationType
-import com.linkedin.drelephant.spark.data._
-import SparkExecutorData.ExecutorInfo
-import SparkJobProgressData.JobInfo
-import org.apache.spark.scheduler.{StageInfo, ApplicationEventListener}
-import org.apache.spark.storage.{StorageStatusTrackingListener, StorageStatus, RDDInfo, StorageStatusListener}
-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
-
-import java.util.{Set => JSet}
-import java.util.{HashSet => JHashSet}
-import java.util.{List => JList}
-import java.util.{ArrayList => JArrayList}
-import java.util.Properties
-
-import scala.collection.mutable
-
-
-/**
- * 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(applicationEventListener: ApplicationEventListener,
- jobProgressListener: JobProgressListener,
- storageStatusListener: StorageStatusListener,
- environmentListener: EnvironmentListener,
- executorsListener: ExecutorsListener,
- storageListener: StorageListener,
- storageStatusTrackingListener: StorageStatusTrackingListener) extends SparkApplicationData {
- 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;
-
- import SparkDataCollection._
-
-
- 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
- }
-}
-
-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
deleted file mode 100644
index 48e453c8f..000000000
--- a/app/org/apache/spark/deploy/history/SparkFSFetcher.scala
+++ /dev/null
@@ -1,354 +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.deploy.history
-
-
-import java.net.{HttpURLConnection, URL, URI}
-import java.security.PrivilegedAction
-import java.io.{IOException, BufferedInputStream, InputStream}
-import java.{io, util}
-import java.util.ArrayList
-import javax.ws.rs.core.UriBuilder
-import com.linkedin.drelephant.configurations.fetcher.FetcherConfigurationData
-import com.linkedin.drelephant.security.HadoopSecurity
-import com.linkedin.drelephant.spark.data.SparkApplicationData
-import com.linkedin.drelephant.util.{MemoryFormatUtils, Utils}
-import com.linkedin.drelephant.analysis.{ApplicationType, AnalyticJob, ElephantFetcher}
-import org.apache.commons.io.FileUtils
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.{Path, FileSystem}
-import org.apache.hadoop.hdfs.web.WebHdfsFileSystem
-import org.apache.hadoop.security.authentication.client.{AuthenticatedURL, AuthenticationException}
-import org.apache.log4j.Logger
-import org.apache.spark.SparkConf
-import org.apache.spark.scheduler.{EventLoggingListener, ReplayListenerBus, ApplicationEventListener}
-import org.apache.spark.storage.{StorageStatusTrackingListener, StorageStatusListener}
-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.io.CompressionCodec
-import org.codehaus.jackson.JsonNode
-import org.codehaus.jackson.map.ObjectMapper
-
-import scala.collection.mutable.ArrayBuffer
-
-
-/**
- * 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 NAME_SERVICES = "dfs.nameservices";
- val DFS_HA_NAMENODES = "dfs.ha.namenodes";
- val DFS_NAMENODE_HTTP_ADDRESS = "dfs.namenode.http-address";
-
- var confEventLogSizeInMb = defEventLogSizeInMb
- if (fetcherConfData.getParamMap.get(LOG_SIZE_XML_FIELD) != null) {
- val logLimitSize = Utils.getParam(fetcherConfData.getParamMap.get(LOG_SIZE_XML_FIELD), 1)
- if (logLimitSize != null) {
- confEventLogSizeInMb = logLimitSize(0)
- }
- }
- logger.info("The event log limit of Spark application is set to " + confEventLogSizeInMb + " MB")
-
- var confEventLogDir = fetcherConfData.getParamMap.get(LOG_DIR_XML_FIELD)
- if (confEventLogDir == null || confEventLogDir.isEmpty) {
- confEventLogDir = defEventLogDir
- }
- logger.info("The event log directory of Spark application is set to " + confEventLogDir)
-
- private val _sparkConf = new SparkConf()
-
- /* Lazy loading for the log directory is very important. Hadoop Configuration() takes time to load itself to reflect
- * properties in the configuration files. Triggering it too early will sometimes make the configuration object empty.
- */
- private lazy val _logDir: String = {
- val conf = new Configuration()
- val nodeAddress = getNamenodeAddress(conf);
- val hdfsAddress = if (nodeAddress == null) "" else "webhdfs://" + nodeAddress
-
- val uri = new URI(_sparkConf.get("spark.eventLog.dir", confEventLogDir))
- val logDir = hdfsAddress + uri.getPath
- logger.info("Looking for spark logs at logDir: " + logDir)
- logDir
- }
-
- /**
- * Returns the namenode address of the active nameNode
- * @param conf The Hadoop configuration
- * @return The namenode address of the active namenode
- */
- def getNamenodeAddress(conf: Configuration): String = {
-
- // check if the fetcherconf has namenode addresses. There can be multiple addresses and
- // we need to check the active namenode address. If a value is specified in the fetcherconf
- // then the value obtained from hadoop configuration won't be used.
- if (fetcherConfData.getParamMap.get(NAMENODE_ADDRESSES) != null) {
- val nameNodes: Array[String] = fetcherConfData.getParamMap.get(NAMENODE_ADDRESSES).split(",");
- for (nameNode <- nameNodes) {
- if (checkActivation(nameNode)) {
- return nameNode;
- }
- }
- }
-
- // if we couldn't find the namenode address in fetcherconf, try to find it in hadoop configuration.
- var isHAEnabled: Boolean = false;
- if (conf.get(NAME_SERVICES) != null) {
- isHAEnabled = true;
- }
-
- // check if HA is enabled
- if (isHAEnabled) {
- // There can be multiple nameservices separated by ',' in case of HDFS federation. It is not supported right now.
- if (conf.get(NAME_SERVICES).split(",").length > 1) {
- logger.info("Multiple name services found. HDFS federation is not supported right now.")
- return null;
- }
- val nameService: String = conf.get(NAME_SERVICES);
- val nameNodeIdentifier: String = conf.get(DFS_HA_NAMENODES + "." + nameService);
- if (nameNodeIdentifier != null) {
- // there can be multiple namenode identifiers separated by ','
- for (nameNodeIdentifierValue <- nameNodeIdentifier.split(",")) {
- val httpValue = conf.get(DFS_NAMENODE_HTTP_ADDRESS + "." + nameService + "." + nameNodeIdentifierValue);
- if (httpValue != null && checkActivation(httpValue)) {
- logger.info("Active namenode : " + httpValue);
- return httpValue;
- }
- }
- }
- }
-
- // if HA is disabled, return the namenode http-address.
- return conf.get(DFS_NAMENODE_HTTP_ADDRESS);
- }
-
- /**
- * Checks if the namenode specified is active or not
- * @param httpValue The namenode configuration http value
- * @return True if the namenode is active, otherwise false
- */
- def checkActivation(httpValue: String): Boolean = {
- val url: URL = new URL("http://" + httpValue + "/jmx?qry=Hadoop:service=NameNode,name=NameNodeStatus");
- val rootNode: JsonNode = readJsonNode(url);
- val status: String = rootNode.path("beans").get(0).path("State").getValueAsText();
- if (status.equals("active")) {
- return true;
- }
- return false;
- }
-
- /**
- * Returns the jsonNode which is read from the url
- * @param url The url of the server
- * @return The jsonNode parsed from the url
- */
- def readJsonNode(url: URL): JsonNode = {
- val _token: AuthenticatedURL.Token = new AuthenticatedURL.Token();
- val _authenticatedURL: AuthenticatedURL = new AuthenticatedURL();
- val _objectMapper: ObjectMapper = new ObjectMapper();
- val conn: HttpURLConnection = _authenticatedURL.openConnection(url, _token)
- return _objectMapper.readTree(conn.getInputStream)
- }
-
-
- private val _security = new HadoopSecurity()
-
- private def fs: FileSystem = {
-
- // For test purpose, if no host presented, use the local file system.
- if (new URI(_logDir).getHost == null) {
- FileSystem.getLocal(new Configuration())
- } else {
- val filesystem = new WebHdfsFileSystem()
- filesystem.initialize(new URI(_logDir), new Configuration())
- filesystem
- }
- }
-
- def fetchData(analyticJob: AnalyticJob): SparkApplicationData = {
- val appId = analyticJob.getAppId()
- _security.doAs[SparkDataCollection](new PrivilegedAction[SparkDataCollection] {
- override def run(): SparkDataCollection = {
- /* Most of Spark logs will be in directory structure: /LOG_DIR/[application_id].
- *
- * Some logs (Spark 1.3+) are in /LOG_DIR/[application_id].snappy
- *
- * Currently we won't be able to parse them even we manually set up the codec. There is problem
- * in JsonProtocol#sparkEventFromJson that it does not handle unmatched SparkListenerEvent, which means
- * it is only backward compatible but not forward. And switching the dependency to Spark 1.3 will raise more
- * problems due to the fact that we are touching the internal codes.
- *
- * In short, this fetcher only works with Spark <=1.2, and we should switch to JSON endpoints with Spark's
- * future release.
- */
- val replayBus = new ReplayListenerBus()
- val applicationEventListener = new ApplicationEventListener
- val jobProgressListener = new JobProgressListener(new SparkConf())
- val environmentListener = new EnvironmentListener
- val storageStatusListener = new StorageStatusListener
- val executorsListener = new ExecutorsListener(storageStatusListener)
- 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.
- val storageStatusTrackingListener = new StorageStatusTrackingListener()
- replayBus.addListener(storageStatusTrackingListener)
-
- val dataCollection = new SparkDataCollection(applicationEventListener = applicationEventListener,
- jobProgressListener = jobProgressListener,
- environmentListener = environmentListener,
- storageStatusListener = storageStatusListener,
- executorsListener = executorsListener,
- storageListener = storageListener,
- storageStatusTrackingListener = storageStatusTrackingListener)
-
- replayBus.addListener(applicationEventListener)
- replayBus.addListener(jobProgressListener)
- replayBus.addListener(environmentListener)
- replayBus.addListener(storageStatusListener)
- replayBus.addListener(executorsListener)
- replayBus.addListener(storageListener)
-
- val logPath = new Path(_logDir, appId)
- val logInput: InputStream =
- if (isLegacyLogDirectory(logPath)) {
- if (!shouldThrottle(logPath)) {
- openLegacyEventLog(logPath)
- } else {
- null
- }
- } else {
- val sparkLogExt = Option(fetcherConfData.getParamMap.get(SPARK_LOG_EXT)).getOrElse(defSparkLogExt)
- val logFilePath = new Path(logPath + sparkLogExt)
- if (!shouldThrottle(logFilePath)) {
- EventLoggingListener.openEventLog(logFilePath, fs)
- } else {
- null
- }
- }
-
- if (logInput == null) {
- 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 "
- + confEventLogSizeInMb + " MB, the parsing process gets throttled.")
- } else {
- logger.info("Replaying Spark logs for application: " + appId)
-
- replayBus.replay(logInput, logPath.toString(), false)
-
- logger.info("Replay completed for application: " + appId)
- }
-
- dataCollection
- }
- })
- }
-
- /**
- * Checks if the log path stores the legacy event log. (Spark <= 1.2 store an event log in a directory)
- *
- * @param entry The path to check
- * @return true if it is legacy log path, else false
- */
- private def isLegacyLogDirectory(entry: Path): Boolean = fs.exists(entry) && fs.getFileStatus(entry).isDirectory()
-
- /**
- * Opens a legacy log path
- *
- * @param dir The directory to open
- * @return an InputStream
- */
- private def openLegacyEventLog(dir: Path): InputStream = {
- val children = fs.listStatus(dir)
- var eventLogPath: Path = null
- var codecName: Option[String] = None
-
- children.foreach { child =>
- child.getPath().getName() match {
- case name if name.startsWith(LOG_PREFIX) =>
- eventLogPath = child.getPath()
- case codec if codec.startsWith(COMPRESSION_CODEC_PREFIX) =>
- codecName = Some(codec.substring(COMPRESSION_CODEC_PREFIX.length()))
- case _ =>
- }
- }
-
- if (eventLogPath == null) {
- throw new IllegalArgumentException(s"$dir is not a Spark application log directory.")
- }
-
- val codec = try {
- codecName.map { c => CompressionCodec.createCodec(_sparkConf, c) }
- } catch {
- case e: Exception =>
- throw new IllegalArgumentException(s"Unknown compression codec $codecName.")
- }
-
- val in = new BufferedInputStream(fs.open(eventLogPath))
- codec.map(_.compressedInputStream(in)).getOrElse(in)
- }
-
- /**
- * Checks if the log parser should be throttled when the file is too large.
- * Note: the current Spark's implementation of ReplayListenerBus will take more than 80 minutes to read a compressed
- * 500 MB event log file. Allowing such reading might block the entire Dr Elephant thread pool.
- *
- * @param eventLogPath The event log path
- * @return If the event log parsing should be throttled
- */
- private def shouldThrottle(eventLogPath: Path): Boolean = {
- fs.getFileStatus(eventLogPath).getLen() > (confEventLogSizeInMb * FileUtils.ONE_MB)
- }
-
- def getEventLogSize(): Double = {
- confEventLogSizeInMb
- }
-
- def getEventLogDir(): String = {
- confEventLogDir
- }
-
-}
-
-private object SparkFSFetcher {
- private val logger = Logger.getLogger(SparkFSFetcher.getClass)
-
- var defEventLogDir = "/system/spark-history"
- var defEventLogSizeInMb = 100d; // 100MB
- var defSparkLogExt = "_1.snappy"
-
- val LOG_SIZE_XML_FIELD = "event_log_size_limit_in_mb"
- val LOG_DIR_XML_FIELD = "event_log_dir"
-
- // Constants used to parse <= Spark 1.2.0 log directories.
- val LOG_PREFIX = "EVENT_LOG_"
- val COMPRESSION_CODEC_PREFIX = EventLoggingListener.COMPRESSION_CODEC_KEY + "_"
-
- // Param map property names that allow users to configer various aspects of the fetcher
- val NAMENODE_ADDRESSES = "namenode_addresses"
- val SPARK_LOG_EXT = "spark_log_ext"
-}
diff --git a/app/org/apache/spark/storage/StorageStatusTrackingListener.scala b/app/org/apache/spark/storage/StorageStatusTrackingListener.scala
deleted file mode 100644
index 5d30a2887..000000000
--- a/app/org/apache/spark/storage/StorageStatusTrackingListener.scala
+++ /dev/null
@@ -1,110 +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.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/app/views/help/spark/helpConfigurationHeuristic.scala.html b/app/views/help/spark/helpConfigurationHeuristic.scala.html
new file mode 100644
index 000000000..abadd6167
--- /dev/null
+++ b/app/views/help/spark/helpConfigurationHeuristic.scala.html
@@ -0,0 +1,18 @@
+@*
+* 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.
+*@
+The results from this heuristic primarily inform you about key app
+configuration settings, including driver memory, executor cores,
+executor instances, executor memory, and the serializer.
diff --git a/app/views/help/spark/helpExecutorsHeuristic.scala.html b/app/views/help/spark/helpExecutorsHeuristic.scala.html
new file mode 100644
index 000000000..70084339c
--- /dev/null
+++ b/app/views/help/spark/helpExecutorsHeuristic.scala.html
@@ -0,0 +1,36 @@
+@*
+* 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.
+*@
+This heuristic concerns the distribution (min, 25p, median, 75p,
+max) of key executor metrics including input bytes, shuffle read
+bytes, shuffle write bytes, storage memory used, and task time. The
+max-to-median ratio determines the severity of any particular metric.
+
+Spark application get resources from YARN allocated all at once,
+and don't release these until the application completes. Thus, it's
+important to balance load on the executors to avoid wasting
+resources.
+
+To achieve better load balancing:
+
+
+ - use an appropriate number of partitions (some small multiple of
+ the # of executors) so that there are enough tasks handling those
+ partitions to keep the executors busy
+ - try avoiding key skew; you should know which partitioner you are
+ using and what is the distribution of your keys
+ - consider enabling spark.speculation, so that straggler tasks can
+ be re-launched
+
diff --git a/app/views/help/spark/helpJobsHeuristic.scala.html b/app/views/help/spark/helpJobsHeuristic.scala.html
new file mode 100644
index 000000000..e72005e00
--- /dev/null
+++ b/app/views/help/spark/helpJobsHeuristic.scala.html
@@ -0,0 +1,20 @@
+@*
+* 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.
+*@
+This heuristic reports job failures and high task failure rates for
+each job.
+
+Job/task failures can occur for a number of reasons, so it is
+recommended to look at the YARN application error logs.
diff --git a/app/views/help/spark/helpStagesHeuristic.scala.html b/app/views/help/spark/helpStagesHeuristic.scala.html
new file mode 100644
index 000000000..6af0162d1
--- /dev/null
+++ b/app/views/help/spark/helpStagesHeuristic.scala.html
@@ -0,0 +1,20 @@
+@*
+* 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.
+*@
+This heuristic reports stage failures, high task failure rates for
+each stage, and long average executor runtimes for each stage.
+
+Stage/task failures can occur for a number of reasons, so it is
+recommended to look at the YARN application error logs.
diff --git a/build.sbt b/build.sbt
index 15d9c9ddb..071e15337 100644
--- a/build.sbt
+++ b/build.sbt
@@ -25,7 +25,7 @@ organization := "com.linkedin.drelephant"
javacOptions in Compile ++= Seq("-source", "1.6", "-target", "1.6")
-libraryDependencies ++= dependencies
+libraryDependencies ++= dependencies map { _.excludeAll(exclusionRules: _*) }
// Create a new custom configuration called compileonly
ivyConfigurations += config("compileonly").hide
@@ -34,3 +34,5 @@ ivyConfigurations += config("compileonly").hide
unmanagedClasspath in Compile ++= update.value.select(configurationFilter("compileonly"))
playJavaSettings
+
+scalaVersion := "2.10.4"
diff --git a/project/Dependencies.scala b/project/Dependencies.scala
index a0b6206e0..b53e2de17 100644
--- a/project/Dependencies.scala
+++ b/project/Dependencies.scala
@@ -25,6 +25,8 @@ object Dependencies {
lazy val gsonVersion = "2.2.4"
lazy val guavaVersion = "18.0" // Hadoop defaultly are using guava 11.0, might raise NoSuchMethodException
lazy val jacksonMapperAslVersion = "1.7.3"
+ lazy val jacksonVersion = "2.5.3"
+ lazy val jerseyVersion = "2.24"
lazy val jsoupVersion = "1.7.3"
lazy val mysqlConnectorVersion = "5.1.36"
lazy val oozieClientVersion = "4.2.0"
@@ -61,8 +63,10 @@ object Dependencies {
var requiredDep = Seq(
"com.google.code.gson" % "gson" % gsonVersion,
"com.google.guava" % "guava" % guavaVersion,
+ "com.jsuereth" %% "scala-arm" % "1.4",
"commons-codec" % "commons-codec" % commonsCodecVersion,
"commons-io" % "commons-io" % commonsIoVersion,
+ "javax.ws.rs" % "javax.ws.rs-api" % "2.0.1",
"mysql" % "mysql-connector-java" % mysqlConnectorVersion,
"org.apache.hadoop" % "hadoop-auth" % hadoopVersion % "compileonly",
"org.apache.hadoop" % "hadoop-mapreduce-client-core" % hadoopVersion % "compileonly",
@@ -74,12 +78,26 @@ object Dependencies {
"org.apache.oozie" % "oozie-client" % oozieClientVersion excludeAll(
ExclusionRule(organization = "org.apache.hadoop")
),
+ "org.glassfish.jersey.core" % "jersey-client" % jerseyVersion,
+ "org.glassfish.jersey.core" % "jersey-common" % jerseyVersion,
+ "org.glassfish.jersey.media" % "jersey-media-json-jackson" % jerseyVersion % Test,
+ "org.glassfish.jersey.test-framework" % "jersey-test-framework-core" % jerseyVersion % Test,
+ "org.glassfish.jersey.test-framework.providers" % "jersey-test-framework-provider-grizzly2" % jerseyVersion % Test,
+ "com.fasterxml.jackson.core" % "jackson-databind" % jacksonVersion,
+ "com.fasterxml.jackson.module" %% "jackson-module-scala" % jacksonVersion,
"io.dropwizard.metrics" % "metrics-core" % "3.1.2",
"io.dropwizard.metrics" % "metrics-healthchecks" % "3.1.2",
"org.mockito" % "mockito-core" % "1.10.19" exclude ("org.hamcrest", "hamcrest-core"),
- "org.jmockit" % "jmockit" % "1.23" % Test
+ "org.jmockit" % "jmockit" % "1.23" % Test,
+ "org.scala-lang.modules" %% "scala-async" % "0.9.5",
+ "org.scalatest" %% "scalatest" % "3.0.0" % Test
) :+ sparkExclusion
var dependencies = Seq(javaJdbc, javaEbean, cache)
dependencies ++= requiredDep
+
+ val exclusionRules = Seq(
+ ExclusionRule(organization = "com.sun.jersey", name = "jersey-core"),
+ ExclusionRule(organization = "com.sun.jersey", name = "jersey-server")
+ )
}
diff --git a/project/build.properties b/project/build.properties
index 58add1433..bb96499e0 100644
--- a/project/build.properties
+++ b/project/build.properties
@@ -14,4 +14,4 @@
# the License.
#
-sbt.version=0.13.0
+sbt.version=0.13.2
\ No newline at end of file
diff --git a/test/com/linkedin/drelephant/analysis/SeverityThresholdsTest.scala b/test/com/linkedin/drelephant/analysis/SeverityThresholdsTest.scala
new file mode 100644
index 000000000..488118e01
--- /dev/null
+++ b/test/com/linkedin/drelephant/analysis/SeverityThresholdsTest.scala
@@ -0,0 +1,73 @@
+/*
+ * 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.analysis
+
+import org.scalatest.{FunSpec, Matchers}
+
+class SeverityThresholdsTest extends FunSpec with Matchers {
+ describe("SeverityThresholds") {
+ it("can be used to represent thresholds considered in ascending order") {
+ val thresholds = SeverityThresholds(low = 0.2D, moderate = 0.4D, severe = 0.6D, critical = 0.8D, ascending = true)
+ thresholds.severityOf(0.1D) should be(Severity.NONE)
+ thresholds.severityOf(0.2D) should be(Severity.LOW)
+ thresholds.severityOf(0.3D) should be(Severity.LOW)
+ thresholds.severityOf(0.4D) should be(Severity.MODERATE)
+ thresholds.severityOf(0.5D) should be(Severity.MODERATE)
+ thresholds.severityOf(0.6D) should be(Severity.SEVERE)
+ thresholds.severityOf(0.7D) should be(Severity.SEVERE)
+ thresholds.severityOf(0.8D) should be(Severity.CRITICAL)
+ thresholds.severityOf(0.9D) should be(Severity.CRITICAL)
+ }
+
+ it("can be used to represent thresholds considered in descending order") {
+ val thresholds = SeverityThresholds(low = 0.8D, moderate = 0.6D, severe = 0.4D, critical = 0.2D, ascending = false)
+ thresholds.severityOf(0.1D) should be(Severity.CRITICAL)
+ thresholds.severityOf(0.2D) should be(Severity.CRITICAL)
+ thresholds.severityOf(0.3D) should be(Severity.SEVERE)
+ thresholds.severityOf(0.4D) should be(Severity.SEVERE)
+ thresholds.severityOf(0.5D) should be(Severity.MODERATE)
+ thresholds.severityOf(0.6D) should be(Severity.MODERATE)
+ thresholds.severityOf(0.7D) should be(Severity.LOW)
+ thresholds.severityOf(0.8D) should be(Severity.LOW)
+ thresholds.severityOf(0.9D) should be(Severity.NONE)
+ }
+
+ it("can be parsed as ascending thresholds from a string that can be processed by Utils.getParam") {
+ SeverityThresholds.parse("0.2,0.4,0.6,0.8", ascending = true) should be(
+ Some(SeverityThresholds(low = 0.2D, moderate = 0.4D, severe = 0.6D, critical = 0.8D, ascending = true))
+ )
+ }
+
+ it("can be parsed as descending thresholds from a string that can be processed by Utils.getParam") {
+ SeverityThresholds.parse("0.8,0.6,0.4,0.2", ascending = false) should be(
+ Some(SeverityThresholds(low = 0.8D, moderate = 0.6D, severe = 0.4D, critical = 0.2D, ascending = false))
+ )
+ }
+
+ it("cannot be created as ascending thresholds with unordered values") {
+ an[IllegalArgumentException] should be thrownBy(
+ SeverityThresholds(low = 0.8D, moderate = 0.6D, severe = 0.4D, critical = 0.2D, ascending = true)
+ )
+ }
+
+ it("cannot be created as descending thresholds with unordered values") {
+ an[IllegalArgumentException] should be thrownBy(
+ SeverityThresholds(low = 0.2D, moderate = 0.4D, severe = 0.6D, critical = 0.8D, ascending = false)
+ )
+ }
+ }
+}
diff --git a/test/com/linkedin/drelephant/configurations/fetcher/FetcherConfigurationTest.java b/test/com/linkedin/drelephant/configurations/fetcher/FetcherConfigurationTest.java
index be3f6bbcd..2db6431c4 100644
--- a/test/com/linkedin/drelephant/configurations/fetcher/FetcherConfigurationTest.java
+++ b/test/com/linkedin/drelephant/configurations/fetcher/FetcherConfigurationTest.java
@@ -127,10 +127,6 @@ public void testParseFetcherConf5() {
FetcherConfiguration fetcherConf = new FetcherConfiguration(document5.getDocumentElement());
assertEquals(fetcherConf.getFetchersConfigurationData().size(), 1);
assertEquals(fetcherConf.getFetchersConfigurationData().get(0).getAppType().getName(), spark);
- assertEquals(fetcherConf.getFetchersConfigurationData().get(0).getParamMap().size(), 3);
- assertEquals(fetcherConf.getFetchersConfigurationData().get(0).getParamMap().get(logSizeField), logSizeValue);
- assertEquals(fetcherConf.getFetchersConfigurationData().get(0).getParamMap().get(logDirField), logDirValue);
}
}
-
diff --git a/test/com/linkedin/drelephant/math/StatisticsTest.java b/test/com/linkedin/drelephant/math/StatisticsTest.java
index 3800b2f86..9c28bf56b 100644
--- a/test/com/linkedin/drelephant/math/StatisticsTest.java
+++ b/test/com/linkedin/drelephant/math/StatisticsTest.java
@@ -98,6 +98,14 @@ public void testPercentile() {
assertEquals(10, Statistics.percentile(finishTimes,10));
assertEquals(10, Statistics.percentile(finishTimes,4));
+ List oddLengthValues = new ArrayList();
+ oddLengthValues.add(1L);
+ oddLengthValues.add(2L);
+ oddLengthValues.add(3L);
+ oddLengthValues.add(4L);
+ oddLengthValues.add(5L);
+ assertEquals(3L, Statistics.percentile(oddLengthValues, 50));
+
List finishTimeSingle = new ArrayList();
finishTimeSingle.add(10L);
assertEquals(10,Statistics.percentile(finishTimeSingle,100));
diff --git a/test/com/linkedin/drelephant/spark/MockSparkApplicationData.java b/test/com/linkedin/drelephant/spark/MockSparkApplicationData.java
deleted file mode 100644
index 40aeacd05..000000000
--- a/test/com/linkedin/drelephant/spark/MockSparkApplicationData.java
+++ /dev/null
@@ -1,98 +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;
-
-import com.linkedin.drelephant.analysis.ApplicationType;
-import com.linkedin.drelephant.spark.data.SparkApplicationData;
-import com.linkedin.drelephant.spark.data.SparkEnvironmentData;
-import com.linkedin.drelephant.spark.data.SparkExecutorData;
-import com.linkedin.drelephant.spark.data.SparkGeneralData;
-import com.linkedin.drelephant.spark.data.SparkJobProgressData;
-import com.linkedin.drelephant.spark.data.SparkStorageData;
-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/SparkMetricsAggregatorTest.scala b/test/com/linkedin/drelephant/spark/SparkMetricsAggregatorTest.scala
new file mode 100644
index 000000000..035842107
--- /dev/null
+++ b/test/com/linkedin/drelephant/spark/SparkMetricsAggregatorTest.scala
@@ -0,0 +1,169 @@
+/*
+ * 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
+
+import java.util.Date
+
+import scala.collection.JavaConverters
+
+import com.linkedin.drelephant.analysis.ApplicationType
+import com.linkedin.drelephant.configurations.aggregator.AggregatorConfigurationData
+import com.linkedin.drelephant.spark.data.{SparkApplicationData, SparkLogDerivedData, SparkRestDerivedData}
+import com.linkedin.drelephant.spark.fetchers.statusapiv1.{ApplicationAttemptInfo, ApplicationInfo, ExecutorSummary}
+import org.apache.spark.scheduler.SparkListenerEnvironmentUpdate
+import org.scalatest.{FunSpec, Matchers}
+
+class SparkMetricsAggregatorTest extends FunSpec with Matchers {
+ import SparkMetricsAggregatorTest._
+
+ describe("SparkMetricsAggregator") {
+ val aggregatorConfigurationData = newFakeAggregatorConfigurationData(
+ Map("allocated_memory_waste_buffer_percentage" -> "0.5")
+ )
+
+ val appId = "application_1"
+
+ 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 = {
+ val executorSummaries = Seq(
+ newFakeExecutorSummary(id = "1", totalDuration = 1000000L),
+ newFakeExecutorSummary(id = "2", totalDuration = 3000000L)
+ )
+ SparkRestDerivedData(
+ applicationInfo,
+ jobDatas = Seq.empty,
+ stageDatas = Seq.empty,
+ executorSummaries
+ )
+ }
+
+ describe("when it has log-derived data") {
+ val logDerivedData = {
+ val environmentUpdate = newFakeSparkListenerEnvironmentUpdate(
+ Map(
+ "spark.serializer" -> "org.apache.spark.serializer.KryoSerializer",
+ "spark.storage.memoryFraction" -> "0.3",
+ "spark.driver.memory" -> "2G",
+ "spark.executor.instances" -> "2",
+ "spark.executor.memory" -> "4g",
+ "spark.shuffle.memoryFraction" -> "0.5"
+ )
+ )
+ SparkLogDerivedData(environmentUpdate)
+ }
+
+ val data = SparkApplicationData(appId, restDerivedData, Some(logDerivedData))
+
+ val aggregator = new SparkMetricsAggregator(aggregatorConfigurationData)
+ aggregator.aggregate(data)
+
+ val result = aggregator.getResult
+
+ it("calculates resources used") {
+ val executorMemoryMb = 4096
+ val totalExecutorTaskTimeSeconds = 1000 + 3000
+ result.getResourceUsed should be(executorMemoryMb * totalExecutorTaskTimeSeconds)
+ }
+
+ it("calculates resources wasted") {
+ val totalExecutorMemoryMb = 2 * 4096
+ val applicationDurationSeconds = 8000
+
+ val executorMemoryMb = 4096
+ val totalExecutorTaskTimeSeconds = 1000 + 3000
+
+ result.getResourceWasted should be(4096 * 4000)
+ }
+
+ it("doesn't calculate total delay") {
+ result.getTotalDelay should be(0L)
+ }
+ }
+
+ describe("when it doesn't have log-derived data") {
+ val data = SparkApplicationData(appId, restDerivedData, logDerivedData = None)
+
+ val aggregator = new SparkMetricsAggregator(aggregatorConfigurationData)
+ aggregator.aggregate(data)
+
+ val result = aggregator.getResult
+
+ it("doesn't calculate resources used") {
+ result.getResourceUsed should be(0L)
+ }
+
+ it("doesn't calculate resources wasted") {
+ result.getResourceWasted should be(0L)
+ }
+
+ it("doesn't calculate total delay") {
+ result.getTotalDelay should be(0L)
+ }
+ }
+ }
+}
+
+object SparkMetricsAggregatorTest {
+ import JavaConverters._
+
+ def newFakeAggregatorConfigurationData(params: Map[String, String] = Map.empty): AggregatorConfigurationData =
+ new AggregatorConfigurationData("org.apache.spark.SparkMetricsAggregator", new ApplicationType("SPARK"), params.asJava)
+
+ def newFakeSparkListenerEnvironmentUpdate(appConfigurationProperties: Map[String, String]): SparkListenerEnvironmentUpdate =
+ SparkListenerEnvironmentUpdate(Map("Spark Properties" -> appConfigurationProperties.toSeq))
+
+ def newFakeApplicationAttemptInfo(
+ attemptId: Option[String],
+ startTime: Date,
+ endTime: Date
+ ): ApplicationAttemptInfo = new ApplicationAttemptInfo(
+ attemptId,
+ startTime,
+ endTime,
+ sparkUser = "foo",
+ completed = true
+ )
+
+ def newFakeExecutorSummary(
+ id: String,
+ totalDuration: Long
+ ): ExecutorSummary = new ExecutorSummary(
+ id,
+ hostPort = "",
+ rddBlocks = 0,
+ memoryUsed = 0,
+ diskUsed = 0,
+ activeTasks = 0,
+ failedTasks = 0,
+ completedTasks = 0,
+ totalTasks = 0,
+ totalDuration,
+ totalInputBytes = 0,
+ totalShuffleRead = 0,
+ totalShuffleWrite = 0,
+ maxMemory = 0,
+ executorLogs = Map.empty
+ )
+}
diff --git a/test/com/linkedin/drelephant/spark/TestSparkAggregatedMetrics.java b/test/com/linkedin/drelephant/spark/TestSparkAggregatedMetrics.java
deleted file mode 100644
index ae4616d86..000000000
--- a/test/com/linkedin/drelephant/spark/TestSparkAggregatedMetrics.java
+++ /dev/null
@@ -1,94 +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;
-
-import com.linkedin.drelephant.analysis.ApplicationType;
-import com.linkedin.drelephant.analysis.HadoopApplicationData;
-import com.linkedin.drelephant.configurations.aggregator.AggregatorConfigurationData;
-import com.linkedin.drelephant.spark.data.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/com/linkedin/drelephant/spark/data/SparkApplicationDataTest.scala b/test/com/linkedin/drelephant/spark/data/SparkApplicationDataTest.scala
new file mode 100644
index 000000000..5cd686bb5
--- /dev/null
+++ b/test/com/linkedin/drelephant/spark/data/SparkApplicationDataTest.scala
@@ -0,0 +1,82 @@
+/*
+ * 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.data
+
+import java.util.Date
+
+import scala.collection.JavaConverters
+
+import com.linkedin.drelephant.spark.fetchers.statusapiv1.{ApplicationAttemptInfo, ApplicationInfo}
+import org.apache.spark.scheduler.SparkListenerEnvironmentUpdate
+import org.scalatest.{FunSpec, Matchers}
+
+class SparkApplicationDataTest extends FunSpec with Matchers {
+ import SparkApplicationDataTest._
+ import JavaConverters._
+
+ describe("SparkApplicationData") {
+ val appId = "application_1"
+ val attemptId = Some("1")
+
+ val applicationAttemptInfo = {
+ val now = System.currentTimeMillis
+ val duration = 8000000L
+ newFakeApplicationAttemptInfo(attemptId, startTime = new Date(now - duration), endTime = new Date(now))
+ }
+
+ val restDerivedData = SparkRestDerivedData(
+ new ApplicationInfo(appId, "app", Seq(applicationAttemptInfo)),
+ jobDatas = Seq.empty,
+ stageDatas = Seq.empty,
+ executorSummaries = Seq.empty
+ )
+
+ val configurationProperties = Map(
+ "spark.serializer" -> "org.apache.spark.serializer.KryoSerializer",
+ "spark.storage.memoryFraction" -> "0.3",
+ "spark.driver.memory" -> "2G",
+ "spark.executor.instances" -> "900",
+ "spark.executor.memory" -> "1g",
+ "spark.shuffle.memoryFraction" -> "0.5"
+ )
+
+ val logDerivedData = SparkLogDerivedData(
+ SparkListenerEnvironmentUpdate(Map("Spark Properties" -> configurationProperties.toSeq))
+ )
+
+ describe(".getConf") {
+ it("returns the Spark properties") {
+ val data = SparkApplicationData(appId, restDerivedData, Some(logDerivedData))
+ data.getConf.asScala should contain theSameElementsAs(configurationProperties)
+ }
+ }
+ }
+}
+
+object SparkApplicationDataTest {
+ def newFakeApplicationAttemptInfo(
+ attemptId: Option[String],
+ startTime: Date,
+ endTime: Date
+ ): ApplicationAttemptInfo = new ApplicationAttemptInfo(
+ attemptId,
+ startTime,
+ endTime,
+ sparkUser = "foo",
+ completed = true
+ )
+}
diff --git a/test/com/linkedin/drelephant/spark/fetchers/SparkFetcherTest.scala b/test/com/linkedin/drelephant/spark/fetchers/SparkFetcherTest.scala
new file mode 100644
index 000000000..7d37168a6
--- /dev/null
+++ b/test/com/linkedin/drelephant/spark/fetchers/SparkFetcherTest.scala
@@ -0,0 +1,215 @@
+/*
+ * 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.fetchers
+
+import java.io.{File, FileOutputStream, InputStream, OutputStream}
+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 org.apache.spark.SparkConf
+import org.apache.spark.scheduler.SparkListenerEnvironmentUpdate
+import org.mockito.Mockito
+import org.scalatest.{FunSpec, Matchers}
+
+class SparkFetcherTest extends FunSpec with Matchers {
+ import SparkFetcherTest._
+
+ describe("SparkFetcher") {
+ import ExecutionContext.Implicits.global
+
+ val fetcherConfigurationData = newFakeFetcherConfigurationData()
+
+ val appId = "application_1"
+
+ val t2 = System.currentTimeMillis
+ val t1 = t2 - 1
+ val duration = 8000000L
+
+ val restDerivedData = SparkRestDerivedData(
+ new ApplicationInfo(
+ appId,
+ "app",
+ Seq(
+ newFakeApplicationAttemptInfo(Some("2"), startTime = new Date(t2 - duration), endTime = new Date(t2)),
+ newFakeApplicationAttemptInfo(Some("1"), startTime = new Date(t1 - duration), endTime = new Date(t1))
+ )
+ ),
+ jobDatas = Seq.empty,
+ stageDatas = Seq.empty,
+ executorSummaries = Seq.empty
+ )
+
+ val logDerivedData = SparkLogDerivedData(SparkListenerEnvironmentUpdate(Map.empty))
+
+ val analyticJob = new AnalyticJob().setAppId(appId)
+
+ it("returns data") {
+ 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)))
+ }
+ val data = sparkFetcher.fetchData(analyticJob)
+ data.appId should be(appId)
+ }
+
+ it("throws an exception if the REST client fails") {
+ 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)))
+ }
+
+ an[Exception] should be thrownBy { sparkFetcher.fetchData(analyticJob) }
+ }
+
+ it("throws an exception if the log client fails") {
+ 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 { throw new Exception() }))
+ }
+
+ an[Exception] should be thrownBy { sparkFetcher.fetchData(analyticJob) }
+ }
+
+ it("gets its SparkConf when SPARK_CONF_DIR is set") {
+ val tempDir = Files.createTempDir()
+
+ 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 fetcherConfigurationData = newFakeFetcherConfigurationData()
+ val sparkFetcher = new SparkFetcher(fetcherConfigurationData) {
+ override lazy val sparkUtils = new SparkUtils() {
+ override val defaultEnv = Map("SPARK_CONF_DIR" -> tempDir.toString)
+ }
+ }
+ val sparkConf = sparkFetcher.sparkConf
+
+ tempDir.delete()
+
+ sparkConf.get("spark.yarn.historyServer.address") should be("jh1.grid.example.com:18080")
+ sparkConf.get("spark.eventLog.enabled") should be("true")
+ sparkConf.get("spark.eventLog.compress") should be("true")
+ sparkConf.get("spark.eventLog.dir") should be("hdfs://nn1.grid.example.com:9000/logs/spark")
+ }
+
+ it("gets its SparkConf when SPARK_HOME is set") {
+ val tempDir = Files.createTempDir()
+ val tempConfDir = new File(tempDir, "conf")
+ tempConfDir.mkdir()
+
+ 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 fetcherConfigurationData = newFakeFetcherConfigurationData()
+ val sparkFetcher = new SparkFetcher(fetcherConfigurationData) {
+ override lazy val sparkUtils = new SparkUtils() {
+ override val defaultEnv = Map("SPARK_HOME" -> tempDir.toString)
+ }
+ }
+ val sparkConf = sparkFetcher.sparkConf
+
+ tempDir.delete()
+
+ sparkConf.get("spark.yarn.historyServer.address") should be("jh1.grid.example.com:18080")
+ sparkConf.get("spark.eventLog.enabled") should be("true")
+ sparkConf.get("spark.eventLog.compress") should be("true")
+ sparkConf.get("spark.eventLog.dir") should be("hdfs://nn1.grid.example.com:9000/logs/spark")
+ }
+
+ 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] }
+ }
+ an[IllegalStateException] should be thrownBy { sparkFetcher.sparkConf }
+ }
+ }
+}
+
+object SparkFetcherTest {
+ import JavaConverters._
+
+ def newFakeFetcherConfigurationData(): FetcherConfigurationData =
+ new FetcherConfigurationData(classOf[SparkFetcher].getName, new ApplicationType("SPARK"), Map.empty.asJava)
+
+ def newFakeApplicationAttemptInfo(
+ attemptId: Option[String],
+ startTime: Date,
+ endTime: Date
+ ): ApplicationAttemptInfo = new ApplicationAttemptInfo(
+ attemptId,
+ startTime,
+ endTime,
+ sparkUser = "foo",
+ completed = true
+ )
+
+ def newFakeSparkRestClient(
+ appId: String,
+ restDerivedData: Future[SparkRestDerivedData]
+ )(
+ implicit ec: ExecutionContext
+ ): SparkRestClient = {
+ val sparkRestClient = Mockito.mock(classOf[SparkRestClient])
+ Mockito.when(sparkRestClient.fetchData(appId)).thenReturn(restDerivedData)
+ sparkRestClient
+ }
+
+ def newFakeSparkLogClient(
+ appId: String,
+ attemptId: Option[String],
+ logDerivedData: Future[SparkLogDerivedData]
+ )(
+ implicit ec: ExecutionContext
+ ): SparkLogClient = {
+ val sparkLogClient = Mockito.mock(classOf[SparkLogClient])
+ 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
new file mode 100644
index 000000000..d5fd38927
--- /dev/null
+++ b/test/com/linkedin/drelephant/spark/fetchers/SparkLogClientTest.scala
@@ -0,0 +1,130 @@
+/*
+ * 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.fetchers
+
+import java.io.{ByteArrayInputStream, ByteArrayOutputStream, InputStream, OutputStream}
+import java.net.URI
+
+import scala.concurrent.ExecutionContext
+
+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.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")
+
+ val sparkConf =
+ new SparkConf()
+ .set("spark.eventLog.dir", "hdfs://nn1.grid.example.com:9000/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 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
+ }
+ }
+
+ sparkLogClient.fetchData(appId, attemptId).map { logDerivedData =>
+ val expectedProperties = Map(
+ "spark.serializer" -> "org.apache.spark.serializer.KryoSerializer",
+ "spark.storage.memoryFraction" -> "0.3",
+ "spark.driver.memory" -> "2G",
+ "spark.executor.instances" -> "900",
+ "spark.executor.memory" -> "1g",
+ "spark.shuffle.memoryFraction" -> "0.5"
+ )
+ val actualProperties = logDerivedData.appConfigurationProperties
+ actualProperties should be(expectedProperties)
+ }
+ }
+ }
+}
+
+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
new file mode 100644
index 000000000..7f325739d
--- /dev/null
+++ b/test/com/linkedin/drelephant/spark/fetchers/SparkRestClientTest.scala
@@ -0,0 +1,188 @@
+/*
+ * 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.fetchers
+
+import java.net.URI
+import java.text.SimpleDateFormat
+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.ext.ContextResolver
+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.scalatest.{AsyncFunSpec, Matchers}
+import org.scalatest.compatible.Assertion
+
+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") {
+ import ExecutionContext.Implicits.global
+ val fakeJerseyServer = new FakeJerseyServer() {
+ override def configure(): Application = super.configure() match {
+ case resourceConfig: ResourceConfig =>
+ resourceConfig
+ .register(classOf[FetchDataFixtures.ApiResource])
+ .register(classOf[FetchDataFixtures.ApplicationResource])
+ .register(classOf[FetchDataFixtures.JobsResource])
+ .register(classOf[FetchDataFixtures.StagesResource])
+ .register(classOf[FetchDataFixtures.ExecutorsResource])
+ case config => config
+ }
+ }
+
+ fakeJerseyServer.setUp()
+
+ val historyServerUri = fakeJerseyServer.target.getUri
+
+ val sparkConf = new SparkConf().set("spark.yarn.historyServer.address", s"${historyServerUri.getHost}:${historyServerUri.getPort}")
+ val sparkRestClient = new SparkRestClient(sparkConf)
+
+ sparkRestClient.fetchData(FetchDataFixtures.APP_ID) map { restDerivedData =>
+ restDerivedData.applicationInfo.id should be(FetchDataFixtures.APP_ID)
+ restDerivedData.applicationInfo.name should be(FetchDataFixtures.APP_NAME)
+ restDerivedData.jobDatas should not be(None)
+ restDerivedData.stageDatas should not be(None)
+ restDerivedData.executorSummaries should not be(None)
+ } andThen { case assertion: Try[Assertion] =>
+ fakeJerseyServer.tearDown()
+ assertion
+ }
+ }
+ }
+}
+
+object SparkRestClientTest {
+ class FakeJerseyServer extends JerseyTest {
+ override def configure(): Application = {
+ forceSet(TestProperties.CONTAINER_PORT, "0")
+ enable(TestProperties.LOG_TRAFFIC)
+ enable(TestProperties.DUMP_ENTITY)
+
+ new ResourceConfig()
+ .register(classOf[FakeJerseyObjectMapperProvider])
+ }
+
+ override def configureClient(clientConfig: ClientConfig): Unit = {
+ clientConfig.register(classOf[FakeJerseyObjectMapperProvider])
+ }
+ }
+
+ class FakeJerseyObjectMapperProvider extends ContextResolver[ObjectMapper] {
+ lazy val objectMapper = {
+ val objectMapper = new ObjectMapper()
+ objectMapper.registerModule(DefaultScalaModule)
+ objectMapper.setDateFormat(dateFormat)
+ objectMapper
+ }
+
+ lazy val dateFormat = {
+ val iso8601 = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'GMT'")
+ val cal = Calendar.getInstance(new SimpleTimeZone(0, "GMT"))
+ iso8601.setCalendar(cal)
+ iso8601
+ }
+
+ override def getContext(cls: Class[_]): ObjectMapper = objectMapper
+ }
+
+ object FetchDataFixtures {
+ val APP_ID = "application_1"
+ val APP_NAME = "app"
+
+ @Path("/api/v1")
+ class ApiResource {
+ @Path("applications/{appId}")
+ def getApplication(): ApplicationResource = new ApplicationResource()
+
+ @Path("applications/{appId}/{attemptId}/jobs")
+ def getJobs(): JobsResource = new JobsResource()
+
+ @Path("applications/{appId}/{attemptId}/stages")
+ def getStages(): StagesResource = new StagesResource()
+
+ @Path("applications/{appId}/{attemptId}/executors")
+ def getExecutors(): ExecutorsResource = new ExecutorsResource()
+ }
+
+ @Produces(Array(MediaType.APPLICATION_JSON))
+ class ApplicationResource {
+ @GET
+ def getApplication(@PathParam("appId") appId: String): ApplicationInfo = {
+ val t2 = System.currentTimeMillis
+ val t1 = t2 - 1
+ val duration = 8000000L
+ new ApplicationInfo(
+ APP_ID,
+ APP_NAME,
+ Seq(
+ newFakeApplicationAttemptInfo(Some("2"), startTime = new Date(t2 - duration), endTime = new Date(t2)),
+ newFakeApplicationAttemptInfo(Some("1"), startTime = new Date(t1 - duration), endTime = new Date(t1))
+ )
+ )
+ }
+ }
+
+ @Produces(Array(MediaType.APPLICATION_JSON))
+ class JobsResource {
+ @GET
+ def getJobs(@PathParam("appId") appId: String, @PathParam("attemptId") attemptId: String): Seq[JobData] =
+ if (attemptId == "2") Seq.empty else throw new Exception()
+ }
+
+ @Produces(Array(MediaType.APPLICATION_JSON))
+ class StagesResource {
+ @GET
+ def getStages(@PathParam("appId") appId: String, @PathParam("attemptId") attemptId: String): Seq[StageData] =
+ if (attemptId == "2") Seq.empty else throw new Exception()
+ }
+
+ @Produces(Array(MediaType.APPLICATION_JSON))
+ class ExecutorsResource {
+ @GET
+ def getExecutors(@PathParam("appId") appId: String, @PathParam("attemptId") attemptId: String): Seq[ExecutorSummary] =
+ if (attemptId == "2") Seq.empty else throw new Exception()
+ }
+ }
+
+ def newFakeApplicationAttemptInfo(
+ attemptId: Option[String],
+ startTime: Date,
+ endTime: Date
+ ): ApplicationAttemptInfo = new ApplicationAttemptInfo(
+ attemptId,
+ startTime,
+ endTime,
+ sparkUser = "foo",
+ completed = true
+ )
+}
diff --git a/test/com/linkedin/drelephant/spark/heuristics/BestPropertiesConventionHeuristicTest.java b/test/com/linkedin/drelephant/spark/heuristics/BestPropertiesConventionHeuristicTest.java
deleted file mode 100644
index 2bda9ba15..000000000
--- a/test/com/linkedin/drelephant/spark/heuristics/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.heuristics;
-
-import com.linkedin.drelephant.analysis.ApplicationType;
-import com.linkedin.drelephant.analysis.Severity;
-import com.linkedin.drelephant.spark.MockSparkApplicationData;
-import com.linkedin.drelephant.spark.data.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.heuristics.BestPropertiesConventionHeuristic.SPARK_DRIVER_MEMORY;
-import static com.linkedin.drelephant.spark.heuristics.BestPropertiesConventionHeuristic.SPARK_EXECUTOR_CORES;
-import static com.linkedin.drelephant.spark.heuristics.BestPropertiesConventionHeuristic.SPARK_SERIALIZER;
-import static com.linkedin.drelephant.spark.heuristics.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/heuristics/ConfigurationHeuristicTest.scala b/test/com/linkedin/drelephant/spark/heuristics/ConfigurationHeuristicTest.scala
new file mode 100644
index 000000000..f1e44602f
--- /dev/null
+++ b/test/com/linkedin/drelephant/spark/heuristics/ConfigurationHeuristicTest.scala
@@ -0,0 +1,198 @@
+/*
+ * 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.heuristics
+
+import com.linkedin.drelephant.spark.data.SparkRestDerivedData
+import com.linkedin.drelephant.spark.fetchers.statusapiv1.ApplicationInfo
+import scala.collection.JavaConverters
+
+import com.linkedin.drelephant.analysis.{ApplicationType, Severity}
+import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData
+import com.linkedin.drelephant.spark.data.{SparkApplicationData, SparkLogDerivedData}
+import org.apache.spark.scheduler.SparkListenerEnvironmentUpdate
+import org.scalatest.{FunSpec, Matchers}
+
+
+class ConfigurationHeuristicTest extends FunSpec with Matchers {
+ import ConfigurationHeuristicTest._
+
+ describe("ConfigurationHeuristic") {
+ val heuristicConfigurationData = newFakeHeuristicConfigurationData(
+ Map(
+ "serializer_if_non_null_recommendation" -> "org.apache.spark.serializer.KryoSerializer",
+ "shuffle_manager_if_non_null_recommendation" -> "sort"
+ )
+ )
+
+ val configurationHeuristic = new ConfigurationHeuristic(heuristicConfigurationData)
+
+ describe(".apply") {
+ val configurationProperties = Map(
+ "spark.serializer" -> "org.apache.spark.serializer.KryoSerializer",
+ "spark.storage.memoryFraction" -> "0.3",
+ "spark.driver.memory" -> "2G",
+ "spark.executor.instances" -> "900",
+ "spark.executor.memory" -> "1g",
+ "spark.shuffle.memoryFraction" -> "0.5"
+ )
+
+ val data = newFakeSparkApplicationData(configurationProperties)
+ val heuristicResult = configurationHeuristic.apply(data)
+ val heuristicResultDetails = heuristicResult.getHeuristicResultDetails
+
+ it("returns the severity") {
+ heuristicResult.getSeverity should be(Severity.NONE)
+ }
+
+ it("returns the driver memory") {
+ val details = heuristicResultDetails.get(0)
+ details.getName should include("spark.driver.memory")
+ details.getValue should be("2 GB")
+ }
+
+ it("returns the executor memory") {
+ val details = heuristicResultDetails.get(1)
+ details.getName should include("spark.executor.memory")
+ details.getValue should be("1 GB")
+ }
+
+ it("returns the executor instances") {
+ val details = heuristicResultDetails.get(2)
+ details.getName should include("spark.executor.instances")
+ details.getValue should be("900")
+ }
+
+ it("returns the executor cores") {
+ val details = heuristicResultDetails.get(3)
+ details.getName should include("spark.executor.cores")
+ details.getValue should include("default")
+ }
+
+ it("returns the serializer") {
+ val details = heuristicResultDetails.get(4)
+ details.getName should include("spark.serializer")
+ details.getValue should be("org.apache.spark.serializer.KryoSerializer")
+ }
+ }
+
+ describe(".Evaluator") {
+ import ConfigurationHeuristic.Evaluator
+
+ def newEvaluatorWithConfigurationProperties(configurationProperties: Map[String, String]): Evaluator = {
+ new Evaluator(configurationHeuristic, newFakeSparkApplicationData(configurationProperties))
+ }
+
+ it("has the driver memory bytes when they're present") {
+ val evaluator = newEvaluatorWithConfigurationProperties(Map("spark.driver.memory" -> "2G"))
+ evaluator.driverMemoryBytes should be(Some(2L * 1024 * 1024 * 1024))
+ }
+
+ it("has no driver memory bytes when they're absent") {
+ val evaluator = newEvaluatorWithConfigurationProperties(Map.empty)
+ evaluator.driverMemoryBytes should be(None)
+ }
+
+ it("has the executor memory bytes when they're present") {
+ val evaluator = newEvaluatorWithConfigurationProperties(Map("spark.executor.memory" -> "1g"))
+ evaluator.executorMemoryBytes should be(Some(1L * 1024 * 1024 * 1024))
+ }
+
+ it("has no executor memory bytes when they're absent") {
+ val evaluator = newEvaluatorWithConfigurationProperties(Map.empty)
+ evaluator.executorMemoryBytes should be(None)
+ }
+
+ it("has the executor instances when they're present") {
+ val evaluator = newEvaluatorWithConfigurationProperties(Map("spark.executor.instances" -> "900"))
+ evaluator.executorInstances should be(Some(900))
+ }
+
+ it("has no executor instances when they're absent") {
+ val evaluator = newEvaluatorWithConfigurationProperties(Map.empty)
+ evaluator.executorInstances should be(None)
+ }
+
+ it("has the executor cores when they're present") {
+ val evaluator = newEvaluatorWithConfigurationProperties(Map("spark.executor.cores" -> "2"))
+ evaluator.executorCores should be(Some(2))
+ }
+
+ it("has no executor cores when they're absent") {
+ val evaluator = newEvaluatorWithConfigurationProperties(Map.empty)
+ evaluator.executorCores should be(None)
+ }
+
+ it("has the serializer when it's present") {
+ val evaluator = newEvaluatorWithConfigurationProperties(Map("spark.serializer" -> "org.apache.spark.serializer.KryoSerializer"))
+ evaluator.serializer should be(Some("org.apache.spark.serializer.KryoSerializer"))
+ }
+
+ it("has no serializer when it's absent") {
+ val evaluator = newEvaluatorWithConfigurationProperties(Map.empty)
+ evaluator.serializer should be(None)
+ }
+
+ it("has the severity of the serializer setting when it matches our recommendation") {
+ val evaluator = newEvaluatorWithConfigurationProperties(Map("spark.serializer" -> "org.apache.spark.serializer.KryoSerializer"))
+ evaluator.serializerSeverity should be(Severity.NONE)
+ }
+
+ it("has the severity of the serializer setting when it doesn't match our recommendation and is non-null") {
+ val evaluator = newEvaluatorWithConfigurationProperties(Map("spark.serializer" -> "org.apache.spark.serializer.FooSerializer"))
+ evaluator.serializerSeverity should be(Severity.MODERATE)
+ }
+
+ it("has the severity of the serializer setting when it is null") {
+ val evaluator = newEvaluatorWithConfigurationProperties(Map.empty)
+ evaluator.serializerSeverity should be(Severity.NONE)
+ }
+
+ it("computes the overall severity when there are some issues") {
+ val evaluator = newEvaluatorWithConfigurationProperties(Map("spark.serializer" -> "org.apache.spark.serializer.FooSerializer"))
+ evaluator.severity should be(Severity.MODERATE)
+ }
+
+ it("computes the overall severity when there are no issues") {
+ val evaluator = newEvaluatorWithConfigurationProperties(Map.empty)
+ evaluator.severity should be(Severity.NONE)
+ }
+ }
+ }
+}
+
+object ConfigurationHeuristicTest {
+ import JavaConverters._
+
+ def newFakeHeuristicConfigurationData(params: Map[String, String] = Map.empty): HeuristicConfigurationData =
+ new HeuristicConfigurationData("heuristic", "class", "view", new ApplicationType("type"), params.asJava)
+
+ def newFakeSparkApplicationData(appConfigurationProperties: Map[String, String]): SparkApplicationData = {
+ val logDerivedData = SparkLogDerivedData(
+ SparkListenerEnvironmentUpdate(Map("Spark Properties" -> appConfigurationProperties.toSeq))
+ )
+
+ val appId = "application_1"
+ val restDerivedData = SparkRestDerivedData(
+ new ApplicationInfo(appId, name = "app", Seq.empty),
+ jobDatas = Seq.empty,
+ stageDatas = Seq.empty,
+ executorSummaries = Seq.empty
+ )
+
+ SparkApplicationData(appId, restDerivedData, Some(logDerivedData))
+ }
+}
diff --git a/test/com/linkedin/drelephant/spark/heuristics/ExecutorsHeuristicTest.scala b/test/com/linkedin/drelephant/spark/heuristics/ExecutorsHeuristicTest.scala
new file mode 100644
index 000000000..5189ea7f2
--- /dev/null
+++ b/test/com/linkedin/drelephant/spark/heuristics/ExecutorsHeuristicTest.scala
@@ -0,0 +1,261 @@
+/*
+ * 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.heuristics
+
+import scala.collection.JavaConverters
+
+import com.linkedin.drelephant.analysis.{ApplicationType, Severity, SeverityThresholds}
+import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData
+import com.linkedin.drelephant.spark.data.{SparkApplicationData, SparkLogDerivedData, SparkRestDerivedData}
+import com.linkedin.drelephant.spark.fetchers.statusapiv1.{ApplicationInfo, ExecutorSummary}
+import org.apache.spark.scheduler.SparkListenerEnvironmentUpdate
+import org.scalatest.{FunSpec, Matchers}
+
+
+class ExecutorsHeuristicTest extends FunSpec with Matchers {
+ import ExecutorsHeuristicTest._
+
+ describe("ExecutorsHeuristic") {
+ val heuristicConfigurationData = newFakeHeuristicConfigurationData(
+ Map(
+ "max_to_median_ratio_severity_thresholds" -> "1.414,2,4,16",
+ "ignore_max_bytes_less_than_threshold" -> "4000000",
+ "ignore_max_millis_less_than_threshold" -> "4000001"
+ )
+ )
+ val executorsHeuristic = new ExecutorsHeuristic(heuristicConfigurationData)
+
+ val maxMemory = 5000000L
+
+ val executorSummaries = Seq(
+ newFakeExecutorSummary(
+ id = "1",
+ memoryUsed = 1000000L,
+ totalDuration = 1000001L,
+ totalInputBytes = 1000002L,
+ totalShuffleRead = 1000003L,
+ totalShuffleWrite = 1000004L,
+ maxMemory
+ ),
+ newFakeExecutorSummary(
+ id = "2",
+ memoryUsed = 2000000L,
+ totalDuration = 2000001L,
+ totalInputBytes = 2000002L,
+ totalShuffleRead = 2000003L,
+ totalShuffleWrite = 2000004L,
+ maxMemory
+ ),
+ newFakeExecutorSummary(
+ id = "3",
+ memoryUsed = 3000000L,
+ totalDuration = 3000001L,
+ totalInputBytes = 3000002L,
+ totalShuffleRead = 3000003L,
+ totalShuffleWrite = 3000004L,
+ maxMemory
+ ),
+ newFakeExecutorSummary(
+ id = "4",
+ memoryUsed = 4000000L,
+ totalDuration = 4000001L,
+ totalInputBytes = 4000002L,
+ totalShuffleRead = 4000003L,
+ totalShuffleWrite = 4000004L,
+ maxMemory
+ )
+ )
+
+ describe(".apply") {
+ val data = newFakeSparkApplicationData(executorSummaries)
+ val heuristicResult = executorsHeuristic.apply(data)
+ val heuristicResultDetails = heuristicResult.getHeuristicResultDetails
+
+ it("returns the severity") {
+ heuristicResult.getSeverity should be(Severity.LOW)
+ }
+
+ it("returns the total storage memory allocated") {
+ val details = heuristicResultDetails.get(0)
+ details.getName should include("storage memory allocated")
+ details.getValue should be("19.07 MB")
+ }
+
+ it("returns the total storage memory used") {
+ val details = heuristicResultDetails.get(1)
+ details.getName should include("storage memory used")
+ details.getValue should be("9.54 MB")
+ }
+
+ it("returns the storage memory utilization rate") {
+ val details = heuristicResultDetails.get(2)
+ details.getName should include("storage memory utilization rate")
+ details.getValue should be("0.500")
+ }
+
+ it("returns the distribution of storage memory used among executors") {
+ val details = heuristicResultDetails.get(3)
+ details.getName should include("storage memory used")
+ details.getValue should include regex("976.56 KB.*976.56 KB.*2.38 MB.*2.86 MB.*3.81 MB")
+ }
+
+ it("returns the distribution of task time among executors") {
+ val details = heuristicResultDetails.get(4)
+ details.getName should include("task time")
+ details.getValue should include regex("16 min 40 sec.*16 min 40 sec.*41 min 40 sec.*50 min.*1 hr 6 min 40 sec")
+ }
+
+ it("returns the distribution of input bytes among executors") {
+ val details = heuristicResultDetails.get(5)
+ details.getName should include("input bytes")
+ details.getValue should include regex("976.56 KB.*976.56 KB.*2.38 MB.*2.86 MB.*3.81 MB")
+ }
+
+ it("returns the distribution of shuffle read bytes among executors") {
+ val details = heuristicResultDetails.get(6)
+ details.getName should include("shuffle read bytes")
+ details.getValue should include regex("976.57 KB.*976.57 KB.*2.38 MB.*2.86 MB.*3.81 MB")
+ }
+
+ it("returns the distribution of shuffle write bytes among executors") {
+ val details = heuristicResultDetails.get(7)
+ details.getName should include("shuffle write bytes")
+ details.getValue should include regex("976.57 KB.*976.57 KB.*2.38 MB.*2.86 MB.*3.81 MB")
+ }
+ }
+
+ describe(".Evaluator") {
+ import ExecutorsHeuristic.Evaluator
+ import ExecutorsHeuristic.Distribution
+
+ val data = newFakeSparkApplicationData(executorSummaries)
+ val evaluator = new Evaluator(executorsHeuristic, data)
+
+ it("has the total storage memory allocated") {
+ evaluator.totalStorageMemoryAllocated should be(20000000L)
+ }
+
+ it("has the total storage memory used") {
+ evaluator.totalStorageMemoryUsed should be(10000000L)
+ }
+
+ it("has the storage memory utilization rate") {
+ evaluator.storageMemoryUtilizationRate should be(0.5D)
+ }
+
+ it("has the distribution of storage memory used among executors") {
+ evaluator.storageMemoryUsedDistribution should be(
+ Distribution(1000000L, 1000000L, 2500000L, 3000000L, 4000000L)
+ )
+ }
+
+ it("has the distribution of task time among executors") {
+ evaluator.taskTimeDistribution should be(
+ Distribution(1000001L, 1000001L, 2500001L, 3000001L, 4000001L)
+ )
+ }
+
+ it("has the distribution of input bytes among executors") {
+ evaluator.inputBytesDistribution should be(
+ Distribution(1000002L, 1000002L, 2500002L, 3000002L, 4000002L)
+ )
+ }
+
+ it("has the distribution of shuffle read among executors") {
+ evaluator.shuffleReadBytesDistribution should be(
+ Distribution(1000003L, 1000003L, 2500003L, 3000003L, 4000003L)
+ )
+ }
+
+ it("has the distribution of shuffle write among executors") {
+ evaluator.shuffleWriteBytesDistribution should be(
+ Distribution(1000004L, 1000004L, 2500004L, 3000004L, 4000004L)
+ )
+ }
+
+ it("computes the overall severity") {
+ evaluator.severity should be(Severity.LOW)
+ }
+
+ it("computes the severity of a given distribution, when the max is large enough") {
+ val distribution = Distribution(min = 0L, p25 = 1000L, median = 1000L, p75 = 1000L, max = 16000L)
+ evaluator.severityOfDistribution(distribution, ignoreMaxLessThanThreshold = 16000L) should be(Severity.CRITICAL)
+ }
+
+ it("computes the severity of a given distribution, when the max is not large enough") {
+ val distribution = Distribution(min = 0L, p25 = 1000L, median = 1000L, p75 = 1000L, max = 16000L)
+ evaluator.severityOfDistribution(distribution, ignoreMaxLessThanThreshold = 16001L) should be(Severity.NONE)
+ }
+
+ it("computes the severity of a given distribution, when the median is zero and the max is large enough") {
+ val distribution = Distribution(min = 0L, p25 = 0L, median = 0L, p75 = 0L, max = 16000L)
+ evaluator.severityOfDistribution(distribution, ignoreMaxLessThanThreshold = 16000L) should be(Severity.CRITICAL)
+ }
+
+ it("computes the severity of a given distribution, when the median is zero and the max is not large enough") {
+ val distribution = Distribution(min = 0L, p25 = 0L, median = 0L, p75 = 0L, max = 16000L)
+ evaluator.severityOfDistribution(distribution, ignoreMaxLessThanThreshold = 16001L) should be(Severity.NONE)
+ }
+ }
+ }
+}
+
+object ExecutorsHeuristicTest {
+ import JavaConverters._
+
+ def newFakeHeuristicConfigurationData(params: Map[String, String] = Map.empty): HeuristicConfigurationData =
+ new HeuristicConfigurationData("heuristic", "class", "view", new ApplicationType("type"), params.asJava)
+
+ def newFakeExecutorSummary(
+ id: String,
+ memoryUsed: Long,
+ totalDuration: Long,
+ totalInputBytes: Long,
+ totalShuffleRead: Long,
+ totalShuffleWrite: Long,
+ maxMemory: Long
+ ): ExecutorSummary = new ExecutorSummary(
+ id,
+ hostPort = "",
+ rddBlocks = 0,
+ memoryUsed,
+ diskUsed = 0,
+ activeTasks = 0,
+ failedTasks = 0,
+ completedTasks = 0,
+ totalTasks = 0,
+ totalDuration,
+ totalInputBytes,
+ totalShuffleRead,
+ totalShuffleWrite,
+ maxMemory,
+ executorLogs = Map.empty
+ )
+
+ def newFakeSparkApplicationData(executorSummaries: Seq[ExecutorSummary]): SparkApplicationData = {
+ val appId = "application_1"
+
+ val restDerivedData = SparkRestDerivedData(
+ new ApplicationInfo(appId, name = "app", Seq.empty),
+ jobDatas = Seq.empty,
+ stageDatas = Seq.empty,
+ executorSummaries
+ )
+
+ SparkApplicationData(appId, restDerivedData, logDerivedData = None)
+ }
+}
diff --git a/test/com/linkedin/drelephant/spark/heuristics/JobsHeuristicTest.scala b/test/com/linkedin/drelephant/spark/heuristics/JobsHeuristicTest.scala
new file mode 100644
index 000000000..2a992576f
--- /dev/null
+++ b/test/com/linkedin/drelephant/spark/heuristics/JobsHeuristicTest.scala
@@ -0,0 +1,165 @@
+/*
+ * 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.heuristics
+
+import scala.collection.JavaConverters
+
+import com.linkedin.drelephant.analysis.{ApplicationType, Severity}
+import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData
+import com.linkedin.drelephant.spark.data.{SparkApplicationData, SparkLogDerivedData, SparkRestDerivedData}
+import com.linkedin.drelephant.spark.fetchers.statusapiv1.{ApplicationInfo, JobData}
+import org.apache.spark.JobExecutionStatus
+import org.apache.spark.scheduler.SparkListenerEnvironmentUpdate
+import org.scalatest.{FunSpec, Matchers}
+
+
+class JobsHeuristicTest extends FunSpec with Matchers {
+ import JobsHeuristicTest._
+
+ describe("JobsHeuristic") {
+ val heuristicConfigurationData = newFakeHeuristicConfigurationData(
+ Map(
+ "job_failure_rate_severity_thresholds" -> "0.2,0.4,0.6,0.8",
+ "job_task_failure_rate_severity_thresholds" -> "0.2,0.4,0.6,0.8"
+ )
+ )
+ val jobsHeuristic = new JobsHeuristic(heuristicConfigurationData)
+ val jobDatas = Seq(
+ newFakeJobData(0, "foo", JobExecutionStatus.SUCCEEDED, numCompleteTasks = 10, numFailedTasks = 0),
+ newFakeJobData(1, "bar", JobExecutionStatus.SUCCEEDED, numCompleteTasks = 8, numFailedTasks = 2),
+ newFakeJobData(2, "baz", JobExecutionStatus.SUCCEEDED, numCompleteTasks = 6, numFailedTasks = 4),
+ newFakeJobData(3, "aaa", JobExecutionStatus.FAILED, numCompleteTasks = 4, numFailedTasks = 6),
+ newFakeJobData(4, "zzz", JobExecutionStatus.FAILED, numCompleteTasks = 2, numFailedTasks = 8)
+ )
+
+ describe(".apply") {
+ val data = newFakeSparkApplicationData(jobDatas)
+ val heuristicResult = jobsHeuristic.apply(data)
+ val heuristicResultDetails = heuristicResult.getHeuristicResultDetails
+
+ it("returns the severity") {
+ heuristicResult.getSeverity should be(Severity.CRITICAL)
+ }
+
+ it("returns the number of completed jobs") {
+ heuristicResultDetails.get(0).getValue should be("3")
+ }
+
+ it("returns the number of failed jobs") {
+ heuristicResultDetails.get(1).getValue should be("2")
+ }
+
+ it("returns the list of failed jobs") {
+ heuristicResultDetails.get(2).getValue should be(
+ s"""|job 3, aaa
+ |job 4, zzz""".stripMargin
+ )
+ }
+
+ it("returns the job failure rate") {
+ heuristicResultDetails.get(3).getValue should be("0.400")
+ }
+
+ it("returns the list of jobs with high task failure rates") {
+ heuristicResultDetails.get(4).getValue should be(
+ s"""|job 3, aaa (task failure rate: 0.600)
+ |job 4, zzz (task failure rate: 0.800)""".stripMargin
+ )
+ }
+ }
+
+ describe(".Evaluator") {
+ import JobsHeuristic.Evaluator
+
+ val data = newFakeSparkApplicationData(jobDatas)
+ val evaluator = new Evaluator(jobsHeuristic, data)
+
+ it("has the number of completed jobs") {
+ evaluator.numCompletedJobs should be(3)
+ }
+
+ it("has the number of failed jobs") {
+ evaluator.numFailedJobs should be(2)
+ }
+
+ it("has the list of failed jobs") {
+ val jobIds = evaluator.failedJobs.map { _.jobId }
+ jobIds should contain theSameElementsInOrderAs(Seq(3, 4))
+ }
+
+ it("has the job failure rate") {
+ evaluator.jobFailureRate should be(Some(0.4D))
+ }
+
+ it("has the list of jobs with high task failure rates") {
+ val jobIdsAndTaskFailureRates =
+ evaluator.jobsWithHighTaskFailureRates.map { case (jobData, taskFailureRate) => (jobData.jobId, taskFailureRate) }
+ jobIdsAndTaskFailureRates should contain theSameElementsInOrderAs(Seq((3, 0.6D), (4, 0.8D)))
+ }
+
+ it("computes the overall severity") {
+ evaluator.severity should be(Severity.CRITICAL)
+ }
+ }
+ }
+}
+
+object JobsHeuristicTest {
+ import JavaConverters._
+
+ def newFakeHeuristicConfigurationData(params: Map[String, String] = Map.empty): HeuristicConfigurationData =
+ new HeuristicConfigurationData("heuristic", "class", "view", new ApplicationType("type"), params.asJava)
+
+ def newFakeJobData(
+ jobId: Int,
+ name: String,
+ status: JobExecutionStatus,
+ numCompleteTasks: Int,
+ numFailedTasks: Int
+ ): JobData = new JobData(
+ jobId,
+ name,
+ description = None,
+ submissionTime = None,
+ completionTime = None,
+ stageIds = Seq.empty,
+ jobGroup = None,
+ status,
+ numTasks = numCompleteTasks + numFailedTasks,
+ numActiveTasks = 0,
+ numCompleteTasks,
+ numSkippedTasks = 0,
+ numFailedTasks,
+ numActiveStages = 0,
+ numCompletedStages = 0,
+ numSkippedStages = 0,
+ numFailedStages = 0
+ )
+
+ def newFakeSparkApplicationData(jobDatas: Seq[JobData]): SparkApplicationData = {
+ val appId = "application_1"
+
+ val restDerivedData = SparkRestDerivedData(
+ new ApplicationInfo(appId, name = "app", Seq.empty),
+ jobDatas,
+ stageDatas = Seq.empty,
+ executorSummaries = Seq.empty
+ )
+
+ SparkApplicationData(appId, restDerivedData, logDerivedData = None)
+ }
+}
diff --git a/test/com/linkedin/drelephant/spark/heuristics/MemoryLimitHeuristicTest.java b/test/com/linkedin/drelephant/spark/heuristics/MemoryLimitHeuristicTest.java
deleted file mode 100644
index c73ed33ab..000000000
--- a/test/com/linkedin/drelephant/spark/heuristics/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.heuristics;
-
-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.MockSparkApplicationData;
-import com.linkedin.drelephant.spark.data.SparkApplicationData;
-import com.linkedin.drelephant.spark.data.SparkEnvironmentData;
-import com.linkedin.drelephant.spark.data.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.data.SparkExecutorData.EXECUTOR_DRIVER_NAME;
-import static com.linkedin.drelephant.spark.heuristics.MemoryLimitHeuristic.SPARK_DRIVER_MEMORY;
-import static com.linkedin.drelephant.spark.heuristics.MemoryLimitHeuristic.SPARK_EXECUTOR_INSTANCES;
-import static com.linkedin.drelephant.spark.heuristics.MemoryLimitHeuristic.SPARK_EXECUTOR_MEMORY;
-import static com.linkedin.drelephant.spark.heuristics.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/spark/heuristics/StagesHeuristicTest.scala b/test/com/linkedin/drelephant/spark/heuristics/StagesHeuristicTest.scala
new file mode 100644
index 000000000..6794df530
--- /dev/null
+++ b/test/com/linkedin/drelephant/spark/heuristics/StagesHeuristicTest.scala
@@ -0,0 +1,191 @@
+/*
+ * 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.heuristics
+
+import scala.collection.JavaConverters
+import scala.concurrent.duration.Duration
+
+import com.linkedin.drelephant.analysis.{ApplicationType, Severity}
+import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData
+import com.linkedin.drelephant.spark.data.{SparkApplicationData, SparkLogDerivedData, SparkRestDerivedData}
+import com.linkedin.drelephant.spark.fetchers.statusapiv1.{ApplicationInfo, JobData, StageData}
+import org.apache.spark.scheduler.SparkListenerEnvironmentUpdate
+import org.apache.spark.status.api.v1.StageStatus
+import org.scalatest.{FunSpec, Matchers}
+
+
+class StagesHeuristicTest extends FunSpec with Matchers {
+ import StagesHeuristicTest._
+
+ describe("StagesHeuristic") {
+ val heuristicConfigurationData = newFakeHeuristicConfigurationData(
+ Map(
+ "stage_failure_rate_severity_thresholds" -> "0.2,0.4,0.6,0.8",
+ "stage_task_failure_rate_severity_thresholds" -> "0.2,0.4,0.6,0.8",
+ "stage_runtime_minutes_severity_thresholds" -> "15,30,45,60"
+ )
+ )
+ val stagesHeuristic = new StagesHeuristic(heuristicConfigurationData)
+ val stageDatas = Seq(
+ newFakeStageData(StageStatus.COMPLETE, 0, numCompleteTasks = 10, numFailedTasks = 0, executorRunTime = Duration("2min").toMillis, "foo"),
+ newFakeStageData(StageStatus.COMPLETE, 1, numCompleteTasks = 8, numFailedTasks = 2, executorRunTime = Duration("2min").toMillis, "bar"),
+ newFakeStageData(StageStatus.COMPLETE, 2, numCompleteTasks = 6, numFailedTasks = 4, executorRunTime = Duration("2min").toMillis, "baz"),
+ newFakeStageData(StageStatus.FAILED, 3, numCompleteTasks = 4, numFailedTasks = 6, executorRunTime = Duration("2min").toMillis, "aaa"),
+ newFakeStageData(StageStatus.FAILED, 4, numCompleteTasks = 2, numFailedTasks = 8, executorRunTime = Duration("2min").toMillis, "zzz"),
+ newFakeStageData(StageStatus.COMPLETE, 5, numCompleteTasks = 10, numFailedTasks = 0, executorRunTime = Duration("0min").toMillis, "bbb"),
+ newFakeStageData(StageStatus.COMPLETE, 6, numCompleteTasks = 10, numFailedTasks = 0, executorRunTime = Duration("30min").toMillis, "ccc"),
+ newFakeStageData(StageStatus.COMPLETE, 7, numCompleteTasks = 10, numFailedTasks = 0, executorRunTime = Duration("60min").toMillis, "ddd"),
+ newFakeStageData(StageStatus.COMPLETE, 8, numCompleteTasks = 10, numFailedTasks = 0, executorRunTime = Duration("90min").toMillis, "eee"),
+ newFakeStageData(StageStatus.COMPLETE, 9, numCompleteTasks = 10, numFailedTasks = 0, executorRunTime = Duration("120min").toMillis, "fff")
+ )
+
+ val appConfigurationProperties = Map("spark.executor.instances" -> "2")
+
+ describe(".apply") {
+ val data = newFakeSparkApplicationData(stageDatas, appConfigurationProperties)
+ val heuristicResult = stagesHeuristic.apply(data)
+ val heuristicResultDetails = heuristicResult.getHeuristicResultDetails
+
+ it("returns the severity") {
+ heuristicResult.getSeverity should be(Severity.CRITICAL)
+ }
+
+ it("returns the number of completed stages") {
+ heuristicResultDetails.get(0).getValue should be("8")
+ }
+
+ it("returns the number of failed stages") {
+ heuristicResultDetails.get(1).getValue should be("2")
+ }
+
+ it("returns the stage failure rate") {
+ heuristicResultDetails.get(2).getValue should be("0.200")
+ }
+
+ it("returns the list of stages with high task failure rates") {
+ heuristicResultDetails.get(3).getValue should be(
+ s"""|stage 3, attempt 0 (task failure rate: 0.600)
+ |stage 4, attempt 0 (task failure rate: 0.800)""".stripMargin
+ )
+ }
+
+ it("returns the list of stages with long runtimes") {
+ heuristicResultDetails.get(4).getValue should be(
+ s"""|stage 8, attempt 0 (runtime: 45 min)
+ |stage 9, attempt 0 (runtime: 1 hr)""".stripMargin
+ )
+ }
+ }
+
+ describe(".Evaluator") {
+ import StagesHeuristic.Evaluator
+
+ val data = newFakeSparkApplicationData(stageDatas, appConfigurationProperties)
+ val evaluator = new Evaluator(stagesHeuristic, data)
+
+ it("has the number of completed stages") {
+ evaluator.numCompletedStages should be(8)
+ }
+
+ it("has the number of failed stages") {
+ evaluator.numFailedStages should be(2)
+ }
+
+ it("has the stage failure rate") {
+ evaluator.stageFailureRate should be(Some(0.2D))
+ }
+
+ it("has the list of stages with high task failure rates") {
+ val stageIdsAndTaskFailureRates =
+ evaluator.stagesWithHighTaskFailureRates.map { case (stageData, taskFailureRate) => (stageData.stageId, taskFailureRate) }
+ stageIdsAndTaskFailureRates should contain theSameElementsInOrderAs(Seq((3, 0.6D), (4, 0.8D)))
+ }
+
+ it("has the list of stages with long average executor runtimes") {
+ val stageIdsAndRuntimes =
+ evaluator.stagesWithLongAverageExecutorRuntimes.map { case (stageData, runtime) => (stageData.stageId, runtime) }
+ stageIdsAndRuntimes should contain theSameElementsInOrderAs(
+ Seq((8, Duration("45min").toMillis), (9, Duration("60min").toMillis))
+ )
+ }
+
+ it("computes the overall severity") {
+ evaluator.severity should be(Severity.CRITICAL)
+ }
+ }
+ }
+}
+
+object StagesHeuristicTest {
+ import JavaConverters._
+
+ def newFakeHeuristicConfigurationData(params: Map[String, String] = Map.empty): HeuristicConfigurationData =
+ new HeuristicConfigurationData("heuristic", "class", "view", new ApplicationType("type"), params.asJava)
+
+ def newFakeStageData(
+ status: StageStatus,
+ stageId: Int,
+ numCompleteTasks: Int,
+ numFailedTasks: Int,
+ executorRunTime: Long,
+ name: String
+ ): StageData = new StageData(
+ status,
+ stageId,
+ attemptId = 0,
+ numActiveTasks = numCompleteTasks + numFailedTasks,
+ numCompleteTasks,
+ numFailedTasks,
+ executorRunTime,
+ inputBytes = 0,
+ inputRecords = 0,
+ outputBytes = 0,
+ outputRecords = 0,
+ shuffleReadBytes = 0,
+ shuffleReadRecords = 0,
+ shuffleWriteBytes = 0,
+ shuffleWriteRecords = 0,
+ memoryBytesSpilled = 0,
+ diskBytesSpilled = 0,
+ name,
+ details = "",
+ schedulingPool = "",
+ accumulatorUpdates = Seq.empty,
+ tasks = None,
+ executorSummary = None
+ )
+
+ def newFakeSparkApplicationData(
+ stageDatas: Seq[StageData],
+ appConfigurationProperties: Map[String, String]
+ ): SparkApplicationData = {
+ val appId = "application_1"
+
+ val restDerivedData = SparkRestDerivedData(
+ new ApplicationInfo(appId, name = "app", Seq.empty),
+ jobDatas = Seq.empty,
+ stageDatas,
+ executorSummaries = Seq.empty
+ )
+
+ val logDerivedData = SparkLogDerivedData(
+ SparkListenerEnvironmentUpdate(Map("Spark Properties" -> appConfigurationProperties.toSeq))
+ )
+
+ SparkApplicationData(appId, restDerivedData, Some(logDerivedData))
+ }
+}
diff --git a/test/com/linkedin/drelephant/util/InfoExtractorTest.java b/test/com/linkedin/drelephant/util/InfoExtractorTest.java
index 87da397d8..6efe52333 100644
--- a/test/com/linkedin/drelephant/util/InfoExtractorTest.java
+++ b/test/com/linkedin/drelephant/util/InfoExtractorTest.java
@@ -26,9 +26,11 @@
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
+import org.junit.runner.RunWith;
import mockit.Expectations;
import mockit.Mocked;
+import mockit.integration.junit4.JMockit;
import org.apache.oozie.client.OozieClient;
import org.apache.oozie.client.WorkflowJob;
@@ -38,6 +40,7 @@
import static org.junit.Assert.assertEquals;
+@RunWith(JMockit.class)
public class InfoExtractorTest {
@Mocked
OozieClient oozieClient;
diff --git a/test/org/apache/spark/deploy/history/DummySparkFSFetcher.scala b/test/org/apache/spark/deploy/history/DummySparkFSFetcher.scala
deleted file mode 100644
index aae10dc3f..000000000
--- a/test/org/apache/spark/deploy/history/DummySparkFSFetcher.scala
+++ /dev/null
@@ -1,62 +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.deploy.history
-
-import java.net.URL;
-import com.linkedin.drelephant.configurations.fetcher.FetcherConfigurationData
-import org.codehaus.jackson.JsonNode
-import org.codehaus.jackson.node.{ObjectNode, JsonNodeFactory}
-
-class DummySparkFSFetcher(val fetcherConfData: FetcherConfigurationData) extends SparkFSFetcher(fetcherConfData) {
-
- override def readJsonNode(url: URL): JsonNode = {
-
- // Create json object of the form:
- // {
- // "beans" : [ {
- // "name" : "Hadoop:service=NameNode,name=NameNodeStatus",
- // "modelerType" : "org.apache.hadoop.hdfs.server.namenode.NameNode",
- // "NNRole" : "NameNode",
- // "HostAndPort" : "sample-sample01-ha2.grid.company.com:9000",
- // "SecurityEnabled" : true,
- // "State" : "active"
- // } ]
- // }
- val nodeFactory: JsonNodeFactory = JsonNodeFactory.instance;
- val node: ObjectNode = nodeFactory.objectNode();
- val child: ObjectNode = nodeFactory.objectNode();
- child.put("name", "Hadoop:service=NameNode, name=NameNodeStatus");
- child.put("modelerType", "org.apache.hadoop.hdfs.server.namenode.NameNode");
- child.put("NNRole", "NameNode");
- child.put("HostAndPort", "sample-sample01-ha2.grid.company.com:9000");
- child.put("SecurityEnabled", "true");
-
- val activeNameNodeUrls = Array(
- "http://sample-ha2.grid.company.com:50070/jmx?qry=Hadoop:service=NameNode,name=NameNodeStatus",
- "http://sample-ha4.grid.company.com:50070/jmx?qry=Hadoop:service=NameNode,name=NameNodeStatus"
- );
- // make ha2,ha4 active and other standby
- if(url.toString().equals(activeNameNodeUrls(0)) || url.toString.equals(activeNameNodeUrls(1))) {
- child.put("State","active")
- } else {
- child.put("State", "standby");
- }
- node.putArray("beans").add(child);
- return node;
- }
-
-}
diff --git a/test/org/apache/spark/deploy/history/SparkDataCollectionTest.java b/test/org/apache/spark/deploy/history/SparkDataCollectionTest.java
deleted file mode 100644
index 0073b868f..000000000
--- a/test/org/apache/spark/deploy/history/SparkDataCollectionTest.java
+++ /dev/null
@@ -1,59 +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.deploy.history;
-
-import com.linkedin.drelephant.spark.data.SparkJobProgressData;
-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() {
- ReplayListenerBus replayBus = new ReplayListenerBus();
- JobProgressListener jobProgressListener = new JobProgressListener(new SparkConf());
-
- replayBus.addListener(jobProgressListener);
-
- SparkDataCollection dataCollection = new SparkDataCollection(null, jobProgressListener,
- null, null, null, null, null);
-
- InputStream in = new BufferedInputStream(
- SparkDataCollectionTest.class.getClassLoader().getResourceAsStream(event_log_dir + "event_log_1"));
- replayBus.replay(in, in.toString(), false);
-
- SparkJobProgressData jobProgressData = dataCollection.getJobProgressData();
- assertNotNull("can't get job progress data", jobProgressData);
- }
-
-}
diff --git a/test/org/apache/spark/deploy/history/SparkFsFetcherTest.java b/test/org/apache/spark/deploy/history/SparkFsFetcherTest.java
deleted file mode 100644
index ec7a8103b..000000000
--- a/test/org/apache/spark/deploy/history/SparkFsFetcherTest.java
+++ /dev/null
@@ -1,219 +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.deploy.history;
-
-import com.linkedin.drelephant.analysis.ElephantFetcher;
-import com.linkedin.drelephant.configurations.fetcher.FetcherConfiguration;
-import com.linkedin.drelephant.configurations.fetcher.FetcherConfigurationData;
-import org.apache.hadoop.fs.Path;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.w3c.dom.Document;
-import org.xml.sax.SAXException;
-import org.apache.spark.deploy.history.DummySparkFSFetcher;
-
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.parsers.ParserConfigurationException;
-import java.io.IOException;
-import java.lang.reflect.InvocationTargetException;
-
-import org.apache.hadoop.conf.Configuration;
-
-import static org.junit.Assert.assertEquals;
-
-public class SparkFsFetcherTest {
-
- private static Document document1 = null;
- private static Document document2 = null;
- private static Document document3 = null;
- private static Document document4 = null;
-
- private static final String spark = "SPARK";
- private static final String defEventLogDir = "/system/spark-history";
- private static final String confEventLogDir = "/custom/configured";
- private static final double defEventLogSize = 100;
- private static final double confEventLogSize = 50;
-
- @BeforeClass
- public static void runBeforeClass() {
- try {
- DocumentBuilderFactory factory = DocumentBuilderFactory.newInstance();
- DocumentBuilder builder = factory.newDocumentBuilder();
- document1 = builder.parse(
- SparkFsFetcherTest.class.getClassLoader().getResourceAsStream(
- "configurations/fetcher/FetcherConfTest5.xml"));
- document2 = builder.parse(
- SparkFsFetcherTest.class.getClassLoader().getResourceAsStream(
- "configurations/fetcher/FetcherConfTest6.xml"));
- document3 = builder.parse(
- SparkFsFetcherTest.class.getClassLoader().getResourceAsStream(
- "configurations/fetcher/FetcherConfTest7.xml"));
- document4 = builder.parse(
- SparkFsFetcherTest.class.getClassLoader().getResourceAsStream(
- "configurations/fetcher/FetcherConfTest8.xml"));
- } catch (ParserConfigurationException e) {
- throw new RuntimeException("XML Parser could not be created.", e);
- } catch (SAXException e) {
- throw new RuntimeException("Test files are not properly formed", e);
- } catch (IOException e) {
- throw new RuntimeException("Unable to read test files ", e);
- }
- }
-
- /**
- * Test for verifying the configured event log directory and log size
- *
- *
- * 50
- * /custom/configured
- *
- */
- @Test
- public void testSparkFetcherConfig() {
- FetcherConfiguration fetcherConf = new FetcherConfiguration(document1.getDocumentElement());
- assertEquals(fetcherConf.getFetchersConfigurationData().size(), 1);
- assertEquals(fetcherConf.getFetchersConfigurationData().get(0).getAppType().getName(), spark);
-
- Class> fetcherClass = null;
- FetcherConfigurationData data = fetcherConf.getFetchersConfigurationData().get(0);
- try {
- fetcherClass = SparkFsFetcherTest.class.getClassLoader().loadClass(data.getClassName());
- Object sparkFetcherInstance = fetcherClass.getConstructor(FetcherConfigurationData.class).newInstance(data);
- if (!(sparkFetcherInstance instanceof ElephantFetcher)) {
- throw new IllegalArgumentException(
- "Class " + fetcherClass.getName() + " is not an implementation of " + ElephantFetcher.class.getName());
- }
-
- // Check if the configurations are picked up correctly
- assertEquals(confEventLogSize, ((SparkFSFetcher) sparkFetcherInstance).getEventLogSize(), 0);
- assertEquals(confEventLogDir, ((SparkFSFetcher) sparkFetcherInstance).getEventLogDir());
-
- } catch (InstantiationException e) {
- throw new RuntimeException("Could not instantiate class " + data.getClassName(), e);
- } catch (ClassNotFoundException e) {
- throw new RuntimeException("Could not find class " + data.getClassName(), e);
- } catch (IllegalAccessException e) {
- throw new RuntimeException("Could not access constructor for class" + data.getClassName(), e);
- } catch (InvocationTargetException e) {
- throw new RuntimeException("Could not invoke class " + data.getClassName(), e);
- } catch (NoSuchMethodException e) {
- throw new RuntimeException("Could not find constructor for class " + data.getClassName(), e);
- }
- }
-
- /**
- * Test for verifying unspecified log directory and log size configs
- *
- *
- *
- */
- @Test
- public void testSparkFetcherUnspecifiedConfig() {
- FetcherConfiguration fetcherConf = new FetcherConfiguration(document3.getDocumentElement());
- assertEquals(fetcherConf.getFetchersConfigurationData().size(), 1);
- assertEquals(fetcherConf.getFetchersConfigurationData().get(0).getAppType().getName(), spark);
-
- Class> fetcherClass = null;
- FetcherConfigurationData data = fetcherConf.getFetchersConfigurationData().get(0);
- try {
- fetcherClass = SparkFsFetcherTest.class.getClassLoader().loadClass(data.getClassName());
- Object sparkFetcherInstance = fetcherClass.getConstructor(FetcherConfigurationData.class).newInstance(data);
- if (!(sparkFetcherInstance instanceof ElephantFetcher)) {
- throw new IllegalArgumentException(
- "Class " + fetcherClass.getName() + " is not an implementation of " + ElephantFetcher.class.getName());
- }
-
- // Check if the default values are used
- assertEquals(defEventLogSize, ((SparkFSFetcher) sparkFetcherInstance).getEventLogSize(), 0);
- assertEquals(defEventLogDir, ((SparkFSFetcher) sparkFetcherInstance).getEventLogDir());
-
- } catch (InstantiationException e) {
- throw new RuntimeException("Could not instantiate class " + data.getClassName(), e);
- } catch (ClassNotFoundException e) {
- throw new RuntimeException("Could not find class " + data.getClassName(), e);
- } catch (IllegalAccessException e) {
- throw new RuntimeException("Could not access constructor for class" + data.getClassName(), e);
- } catch (InvocationTargetException e) {
- throw new RuntimeException("Could not invoke class " + data.getClassName(), e);
- } catch (NoSuchMethodException e) {
- throw new RuntimeException("Could not find constructor for class " + data.getClassName(), e);
- }
- }
-
- /**
- * Test for verifying empty log directory and log size configs
- *
- *
- *
- * /system/spark-history
- *
- */
- @Test
- public void testSparkFetcherEmptyConfig() {
- FetcherConfiguration fetcherConf = new FetcherConfiguration(document2.getDocumentElement());
- assertEquals(fetcherConf.getFetchersConfigurationData().size(), 1);
- assertEquals(fetcherConf.getFetchersConfigurationData().get(0).getAppType().getName(), spark);
-
- Class> fetcherClass = null;
- FetcherConfigurationData data = fetcherConf.getFetchersConfigurationData().get(0);
- try {
- fetcherClass = SparkFsFetcherTest.class.getClassLoader().loadClass(data.getClassName());
- Object sparkFetcherInstance = fetcherClass.getConstructor(FetcherConfigurationData.class).newInstance(data);
- if (!(sparkFetcherInstance instanceof ElephantFetcher)) {
- throw new IllegalArgumentException(
- "Class " + fetcherClass.getName() + " is not an implementation of " + ElephantFetcher.class.getName());
- }
-
- // Check if the default values are used
- assertEquals(defEventLogSize, ((SparkFSFetcher) sparkFetcherInstance).getEventLogSize(), 0);
- assertEquals(defEventLogDir, ((SparkFSFetcher) sparkFetcherInstance).getEventLogDir());
-
- } catch (InstantiationException e) {
- throw new RuntimeException("Could not instantiate class " + data.getClassName(), e);
- } catch (ClassNotFoundException e) {
- throw new RuntimeException("Could not find class " + data.getClassName(), e);
- } catch (IllegalAccessException e) {
- throw new RuntimeException("Could not access constructor for class" + data.getClassName(), e);
- } catch (InvocationTargetException e) {
- throw new RuntimeException("Could not invoke class " + data.getClassName(), e);
- } catch (NoSuchMethodException e) {
- throw new RuntimeException("Could not find constructor for class " + data.getClassName(), e);
- }
- }
-
- // checks the namenode address from the hadoopconf
- @Test
- public void testGetNamenodeAddressFromHadoopConf() {
- FetcherConfiguration fetcherConf = new FetcherConfiguration(document2.getDocumentElement());
- DummySparkFSFetcher fetcher = new DummySparkFSFetcher(fetcherConf.getFetchersConfigurationData().get(0));
- Configuration conf = new Configuration();
- String nameNode = fetcher.getNamenodeAddress(conf);
- assertEquals(nameNode,"sample-ha2.grid.company.com:50070");
- }
-
- // checks the namenode address from fetcherConf
- @Test
- public void testGetNamenodeAddressFromFetcherConf() {
- FetcherConfiguration fetcherConf = new FetcherConfiguration(document4.getDocumentElement());
- DummySparkFSFetcher fetcher = new DummySparkFSFetcher(fetcherConf.getFetchersConfigurationData().get(0));
- Configuration conf = new Configuration();
- String nameNode = fetcher.getNamenodeAddress(conf);
- assertEquals(nameNode,"sample-ha4.grid.company.com:50070");
- }
-}
-
diff --git a/test/resources/configurations/fetcher/FetcherConfTest5.xml b/test/resources/configurations/fetcher/FetcherConfTest5.xml
index 4ad18425b..2372f0828 100644
--- a/test/resources/configurations/fetcher/FetcherConfTest5.xml
+++ b/test/resources/configurations/fetcher/FetcherConfTest5.xml
@@ -18,10 +18,8 @@
spark
- org.apache.spark.deploy.history.SparkFSFetcher
+ com.linkedin.drelephant.spark.fetchers.SparkFetcher
- 50
- /custom/configured
diff --git a/test/resources/configurations/fetcher/FetcherConfTest6.xml b/test/resources/configurations/fetcher/FetcherConfTest6.xml
deleted file mode 100644
index 0adb493f0..000000000
--- a/test/resources/configurations/fetcher/FetcherConfTest6.xml
+++ /dev/null
@@ -1,27 +0,0 @@
-
-
-
-
-
- spark
- org.apache.spark.deploy.history.SparkFSFetcher
-
-
-
-
-
-
diff --git a/test/resources/configurations/fetcher/FetcherConfTest7.xml b/test/resources/configurations/fetcher/FetcherConfTest7.xml
deleted file mode 100644
index 1564cb313..000000000
--- a/test/resources/configurations/fetcher/FetcherConfTest7.xml
+++ /dev/null
@@ -1,25 +0,0 @@
-
-
-
-
-
- spark
- org.apache.spark.deploy.history.SparkFSFetcher
-
-
-
-
diff --git a/test/resources/configurations/fetcher/FetcherConfTest8.xml b/test/resources/configurations/fetcher/FetcherConfTest8.xml
deleted file mode 100644
index f49103dae..000000000
--- a/test/resources/configurations/fetcher/FetcherConfTest8.xml
+++ /dev/null
@@ -1,26 +0,0 @@
-
-
-
-
-
- spark
- org.apache.spark.deploy.history.SparkFSFetcher
-
- sample-ha3.grid.company.com:50070,sample-ha4.grid.company.com:50070
-
-
-
\ No newline at end of file
diff --git a/test/resources/spark-defaults.conf b/test/resources/spark-defaults.conf
new file mode 100644
index 000000000..de9cc375a
--- /dev/null
+++ b/test/resources/spark-defaults.conf
@@ -0,0 +1,4 @@
+spark.yarn.historyServer.address = jh1.grid.example.com:18080
+spark.eventLog.enabled = true
+spark.eventLog.compress = true
+spark.eventLog.dir = hdfs://nn1.grid.example.com:9000/logs/spark
diff --git a/test/resources/spark_event_logs/event_log_1 b/test/resources/spark_event_logs/event_log_1
deleted file mode 100644
index 0ed97efaf..000000000
--- a/test/resources/spark_event_logs/event_log_1
+++ /dev/null
@@ -1,32 +0,0 @@
-{"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}
diff --git a/test/resources/spark_event_logs/event_log_2 b/test/resources/spark_event_logs/event_log_2
new file mode 100644
index 000000000..35101daa4
--- /dev/null
+++ b/test/resources/spark_event_logs/event_log_2
@@ -0,0 +1,4 @@
+{"Event":"SparkListenerLogStart","Spark Version":"1.4.1"}
+{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"127.0.0.1","Port":54157},"Maximum Memory":515411804,"Timestamp":1475761114342}
+{"Event":"SparkListenerEnvironmentUpdate","Spark Properties":{"spark.serializer":"org.apache.spark.serializer.KryoSerializer","spark.storage.memoryFraction":"0.3","spark.driver.memory":"2G","spark.executor.instances":"900","spark.executor.memory":"1g","spark.shuffle.memoryFraction":"0.5"},"JVM Information":{},"System Properties":{},"Classpath Entries":{}}
+{"Event":"SparkListenerApplicationStart","App Name":"app","App ID":"application_1","Timestamp":1475761112578,"User":"foo","App Attempt ID":"1"}