From 0eb7722da9e84cb8485e081b241d8260eceba971 Mon Sep 17 00:00:00 2001 From: twinkle sachdeva Date: Mon, 2 Feb 2015 21:08:35 +0530 Subject: [PATCH 01/22] SPARK-4705: Doing cherry-pick of fix into master --- .../scala/org/apache/spark/SparkContext.scala | 4 ++- .../scheduler/EventLoggingListener.scala | 27 ++++++++++++++----- .../spark/scheduler/SchedulerBackend.scala | 7 +++++ .../spark/scheduler/TaskScheduler.scala | 8 ++++++ .../spark/scheduler/TaskSchedulerImpl.scala | 2 ++ .../spark/deploy/yarn/ApplicationMaster.scala | 4 +++ .../cluster/YarnClusterSchedulerBackend.scala | 7 ++++- 7 files changed, 50 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 3f1a7dd99d635..9c3f0eeea4ed4 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -386,6 +386,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli taskScheduler.start() val applicationId: String = taskScheduler.applicationId() + val applicationAttemptId : String = taskScheduler.applicationAttemptId() conf.set("spark.app.id", applicationId) env.blockManager.initialize(applicationId) @@ -402,7 +403,8 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli private[spark] val eventLogger: Option[EventLoggingListener] = { if (isEventLogEnabled) { val logger = - new EventLoggingListener(applicationId, eventLogDir.get, conf, hadoopConfiguration) + new EventLoggingListener(applicationId, applicationAttemptId, + eventLogDir.get, conf, hadoopConfiguration) logger.start() listenerBus.addListener(logger) Some(logger) diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 08e7727db2fde..02a2263940821 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -47,6 +47,7 @@ import org.apache.spark.util.{JsonProtocol, Utils} */ private[spark] class EventLoggingListener( appId: String, + appAttemptId : String, logBaseDir: URI, sparkConf: SparkConf, hadoopConf: Configuration) @@ -55,7 +56,7 @@ private[spark] class EventLoggingListener( import EventLoggingListener._ def this(appId: String, logBaseDir: URI, sparkConf: SparkConf) = - this(appId, logBaseDir, sparkConf, SparkHadoopUtil.get.newConfiguration(sparkConf)) + this(appId, "", logBaseDir, sparkConf, SparkHadoopUtil.get.newConfiguration(sparkConf)) private val shouldCompress = sparkConf.getBoolean("spark.eventLog.compress", false) private val shouldOverwrite = sparkConf.getBoolean("spark.eventLog.overwrite", false) @@ -89,7 +90,7 @@ private[spark] class EventLoggingListener( private[scheduler] val loggedEvents = new ArrayBuffer[JValue] // Visible for tests only. - private[scheduler] val logPath = getLogPath(logBaseDir, appId, compressionCodecName) + private[scheduler] val logPath = getLogPath(logBaseDir, appId, compressionCodecName, appAttemptId) /** * Creates the log file in the configured log directory. @@ -254,18 +255,30 @@ private[spark] object EventLoggingListener extends Logging { * * @param logBaseDir Directory where the log file will be written. * @param appId A unique app ID. + * @param appAttemptId A unique attempt id of appId. * @param compressionCodecName Name to identify the codec used to compress the contents * of the log, or None if compression is not enabled. * @return A path which consists of file-system-safe characters. */ def getLogPath( - logBaseDir: URI, + logBaseDir: String, appId: String, + appAttemptId: String, compressionCodecName: Option[String] = None): String = { - val sanitizedAppId = appId.replaceAll("[ :/]", "-").replaceAll("[.${}'\"]", "_").toLowerCase - // e.g. app_123, app_123.lzf - val logName = sanitizedAppId + compressionCodecName.map { "." + _ }.getOrElse("") - logBaseDir.toString.stripSuffix("/") + "/" + logName + val name = appId.replaceAll("[ :/]", "-").replaceAll("[${}'\"]", "_").toLowerCase + + if (appAttemptId.equals("")) { + Utils.resolveURI(logBaseDir) + "/" + name.stripSuffix("/") + } else { + Utils.resolveURI(logBaseDir) + "/" + appAttemptId + "/" + name.stripSuffix("/") + } + } + + def getLogPath( + logBaseDir: String, + appId: String, + compressionCodecName: Option[String] = None): String = { + getLogPath(logBaseDir, appId, "", compressionCodecName) } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala index 992c477493d8e..6a6ab0c82f310 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala @@ -41,4 +41,11 @@ private[spark] trait SchedulerBackend { */ def applicationId(): String = appId + /** + * Get an application ID associated with the job. + * + * @return An application attempt id + */ + def applicationAttemptId(): String = "" + } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index ed3418676e077..d612409c81b6f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -78,4 +78,12 @@ private[spark] trait TaskScheduler { * Process a lost executor */ def executorLost(executorId: String, reason: ExecutorLossReason): Unit + + /** + * Get an application's attempt Id associated with the job. + * + * @return An application's Attempt ID + */ + def applicationAttemptId(): String = "" + } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 076b36e86c0ce..265de3699a822 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -514,6 +514,8 @@ private[spark] class TaskSchedulerImpl( } override def applicationId(): String = backend.applicationId() + + override def applicationAttemptId() : String = backend.applicationAttemptId() } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 24a1e02795218..db5af9d29b99a 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -90,6 +90,10 @@ private[spark] class ApplicationMaster( // Propagate the application ID so that YarnClusterSchedulerBackend can pick it up. System.setProperty("spark.yarn.app.id", appAttemptId.getApplicationId().toString()) + + //Propagate the attempt if, so that in case of event logging, different attempt's logs gets created in different directory + System.setProperty("spark.yarn.app.attemptid", appAttemptId.getAttemptId().toString()) + } logInfo("ApplicationAttemptId: " + appAttemptId) diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala index b1de81e6a8b0f..7662c02c8c529 100644 --- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala @@ -46,5 +46,10 @@ private[spark] class YarnClusterSchedulerBackend( logError("Application ID is not set.") super.applicationId } - + + override def applicationAttemptId(): String = + sc.getConf.getOption("spark.yarn.app.attemptid").getOrElse { + logError("Application attempt ID is not set.") + super.applicationAttemptId + } } From 4c1fc262ea12ff6e4f16fd5ef66db12a7635716d Mon Sep 17 00:00:00 2001 From: twinkle sachdeva Date: Wed, 25 Feb 2015 10:15:43 +0530 Subject: [PATCH 02/22] SPARK-4705 Incorporating the review comments regarding formatting, will do the rest of the changes after this --- core/src/main/scala/org/apache/spark/SparkContext.scala | 7 +++---- .../org/apache/spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../scheduler/cluster/YarnClusterSchedulerBackend.scala | 3 +++ 3 files changed, 7 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 9c3f0eeea4ed4..cee07c3116a5a 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -386,7 +386,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli taskScheduler.start() val applicationId: String = taskScheduler.applicationId() - val applicationAttemptId : String = taskScheduler.applicationAttemptId() + val applicationAttemptId: String = taskScheduler.applicationAttemptId() conf.set("spark.app.id", applicationId) env.blockManager.initialize(applicationId) @@ -402,9 +402,8 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli // Optionally log Spark events private[spark] val eventLogger: Option[EventLoggingListener] = { if (isEventLogEnabled) { - val logger = - new EventLoggingListener(applicationId, applicationAttemptId, - eventLogDir.get, conf, hadoopConfiguration) + val logger = new EventLoggingListener( + applicationId, applicationAttemptId, eventLogDir.get, conf, hadoopConfiguration) logger.start() listenerBus.addListener(logger) Some(logger) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 265de3699a822..674ef4a737a85 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -515,7 +515,7 @@ private[spark] class TaskSchedulerImpl( override def applicationId(): String = backend.applicationId() - override def applicationAttemptId() : String = backend.applicationAttemptId() + override def applicationAttemptId(): String = backend.applicationAttemptId() } diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala index 7662c02c8c529..f20f4dcb00d64 100644 --- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala @@ -48,6 +48,9 @@ private[spark] class YarnClusterSchedulerBackend( } override def applicationAttemptId(): String = + // In YARN Cluster mode, spark.yarn.app.attemptid is expect to be set + // before user application is launched. + // So, if spark.yarn.app.id is not set, it is something wrong. sc.getConf.getOption("spark.yarn.app.attemptid").getOrElse { logError("Application attempt ID is not set.") super.applicationAttemptId From 6b2e521b2b0e0485e41489e1b3471c363d4b999b Mon Sep 17 00:00:00 2001 From: twinkle sachdeva Date: Wed, 25 Feb 2015 14:50:33 +0530 Subject: [PATCH 03/22] SPARK-4705 Incorporating the review comments regarding formatting, will do the rest of the changes after this --- .../scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index db5af9d29b99a..3cc2a9f5570b1 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -91,7 +91,8 @@ private[spark] class ApplicationMaster( // Propagate the application ID so that YarnClusterSchedulerBackend can pick it up. System.setProperty("spark.yarn.app.id", appAttemptId.getApplicationId().toString()) - //Propagate the attempt if, so that in case of event logging, different attempt's logs gets created in different directory + // Propagate the attempt if, so that in case of event logging, + // different attempt's logs gets created in different directory System.setProperty("spark.yarn.app.attemptid", appAttemptId.getAttemptId().toString()) } From 318525ad711d121e7edb4110c8ff433d47d0f59a Mon Sep 17 00:00:00 2001 From: "twinkle.sachdeva" Date: Sun, 1 Mar 2015 20:32:50 +0530 Subject: [PATCH 04/22] SPARK-4705: 1) moved from directory structure to single file, as per the master branch. 2) Added the attempt id inside the SparkListenerApplicationStart, to make the info available independent of directory structure. 3) Changes in History Server to render the UI as per the snaphot II --- .../scala/org/apache/spark/SparkContext.scala | 2 +- .../history/ApplicationHistoryProvider.scala | 3 +- .../deploy/history/FsHistoryProvider.scala | 17 ++- .../spark/deploy/history/HistoryPage.scala | 137 ++++++++++++++++-- .../scheduler/ApplicationEventListener.scala | 2 + .../scheduler/EventLoggingListener.scala | 11 +- .../spark/scheduler/SparkListener.scala | 4 +- .../org/apache/spark/util/JsonProtocol.scala | 6 +- 8 files changed, 157 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index cee07c3116a5a..3ed1f8157e496 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1757,7 +1757,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli // Note: this code assumes that the task scheduler has been initialized and has contacted // the cluster manager to get an application ID (in case the cluster manager provides one). listenerBus.post(SparkListenerApplicationStart(appName, Some(applicationId), - startTime, sparkUser)) + startTime, sparkUser, applicationAttemptId)) } /** Post the application end event */ diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala index ea6c85ee511d5..7509b6d6b9cf7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala @@ -26,7 +26,8 @@ private[history] case class ApplicationHistoryInfo( endTime: Long, lastUpdated: Long, sparkUser: String, - completed: Boolean = false) + completed: Boolean = false, + appAttemptId: String = "") private[history] abstract class ApplicationHistoryProvider { diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 9d40d8c8fd7a8..7e31a299ae925 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -248,7 +248,13 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis if (!mergedApps.contains(info.id) || mergedApps(info.id).logPath.endsWith(EventLoggingListener.IN_PROGRESS) && !info.logPath.endsWith(EventLoggingListener.IN_PROGRESS)) { - mergedApps += (info.id -> info) + val key = + if (info.appAttemptId.equals("")) { + info.id + } else { + info.id + "_" + info.appAttemptId + } + mergedApps += (key -> info) } } @@ -343,7 +349,8 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis appListener.endTime.getOrElse(-1L), getModificationTime(eventLog).get, appListener.sparkUser.getOrElse(NOT_STARTED), - isApplicationCompleted(eventLog)) + isApplicationCompleted(eventLog), + appListener.appAttemptId.getOrElse("")) } finally { logInput.close() } @@ -438,5 +445,7 @@ private class FsApplicationHistoryInfo( endTime: Long, lastUpdated: Long, sparkUser: String, - completed: Boolean = true) - extends ApplicationHistoryInfo(id, name, startTime, endTime, lastUpdated, sparkUser, completed) + completed: Boolean = true, + appAttemptId: String ="") + extends ApplicationHistoryInfo( + id, name, startTime, endTime, lastUpdated, sparkUser, completed, appAttemptId) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index 6e432d63c6b5a..87159715b34e8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -22,6 +22,9 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.ui.{WebUIPage, UIUtils} +import scala.collection.immutable.ListMap +import scala.collection.mutable.HashMap +import scala.collection.mutable.ArrayBuffer private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { @@ -34,18 +37,31 @@ private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") val requestedIncomplete = Option(request.getParameter("showIncomplete")).getOrElse("false").toBoolean - val allApps = parent.getApplicationList().filter(_.completed != requestedIncomplete) - val actualFirst = if (requestedFirst < allApps.size) requestedFirst else 0 - val apps = allApps.slice(actualFirst, Math.min(actualFirst + pageSize, allApps.size)) - + val allCompletedAppsNAttempts = + parent.getApplicationList().filter(_.completed != requestedIncomplete) + val (hasAttemptInfo, appToAttemptMap) = getApplicationLevelList(allCompletedAppsNAttempts) + + val allAppsSize = allCompletedAppsNAttempts.size + + val actualFirst = if (requestedFirst < allAppsSize) requestedFirst else 0 + val apps = + allCompletedAppsNAttempts.slice(actualFirst, Math.min(actualFirst + pageSize, allAppsSize)) + val appWithAttemptsDisplayList = + appToAttemptMap.slice(actualFirst, Math.min(actualFirst + pageSize, allAppsSize)) + val actualPage = (actualFirst / pageSize) + 1 - val last = Math.min(actualFirst + pageSize, allApps.size) - 1 - val pageCount = allApps.size / pageSize + (if (allApps.size % pageSize > 0) 1 else 0) + val last = Math.min(actualFirst + pageSize, allAppsSize) - 1 + val pageCount = allAppsSize / pageSize + (if (allAppsSize % pageSize > 0) 1 else 0) val secondPageFromLeft = 2 val secondPageFromRight = pageCount - 1 - val appTable = UIUtils.listingTable(appHeader, appRow, apps) + val appTable = + if (hasAttemptInfo) { + UIUtils.listingTable(appWithAttemptHeader, appWithAttemptRow, appWithAttemptsDisplayList) + } else { + UIUtils.listingTable(appHeader, appRow, apps) + } val providerConfig = parent.getProviderConfig() val content =
@@ -59,7 +75,7 @@ private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") // to the first and last page. If the current page +/- `plusOrMinus` is greater // than the 2nd page from the first page or less than the 2nd page from the last // page, `...` will be displayed. - if (allApps.size > 0) { + if (allAppsSize > 0) { val leftSideIndices = rangeIndices(actualPage - plusOrMinus until actualPage, 1 < _, requestedIncomplete) val rightSideIndices = @@ -67,7 +83,7 @@ private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") requestedIncomplete)

- Showing {actualFirst + 1}-{last + 1} of {allApps.size} + Showing {actualFirst + 1}-{last + 1} of {allAppsSize} {if (requestedIncomplete) "(Incomplete applications)"} { @@ -113,6 +129,36 @@ private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("")

UIUtils.basicSparkPage(content, "History Server") } + + private def getApplicationLevelList (appNattemptList: Iterable[ApplicationHistoryInfo]) ={ + // Create HashMap as per the multiple attempts for one application. + // If there is no attempt specific stuff, then + // do return false, to indicate the same, so that previous UI gets displayed. + var hasAttemptInfo = false + val appToAttemptInfo = new HashMap[String, ArrayBuffer[ApplicationHistoryInfo]] + for( appAttempt <- appNattemptList) { + if(!appAttempt.appAttemptId.equals("")){ + hasAttemptInfo = true + val attemptId = appAttempt.appAttemptId.toInt + if(appToAttemptInfo.contains(appAttempt.id)){ + val currentAttempts = appToAttemptInfo.get(appAttempt.id).get + currentAttempts += appAttempt + appToAttemptInfo.put( appAttempt.id, currentAttempts) + } else { + val currentAttempts = new ArrayBuffer[ApplicationHistoryInfo]() + currentAttempts += appAttempt + appToAttemptInfo.put( appAttempt.id, currentAttempts ) + } + }else { + val currentAttempts = new ArrayBuffer[ApplicationHistoryInfo]() + currentAttempts += appAttempt + appToAttemptInfo.put(appAttempt.id, currentAttempts) + } + } + val sortedMap = ListMap(appToAttemptInfo.toSeq.sortWith(_._1 > _._1):_*) + (hasAttemptInfo, sortedMap) + } + private val appHeader = Seq( "App ID", @@ -128,6 +174,16 @@ private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") range.filter(condition).map(nextPage => {nextPage} ) } + + private val appWithAttemptHeader = Seq( + "App ID", + "App Name", + "Attempt ID", + "Started", + "Completed", + "Duration", + "Spark User", + "Last Updated") private def appRow(info: ApplicationHistoryInfo): Seq[Node] = { val uiAddress = HistoryServer.UI_PATH_PREFIX + s"/${info.id}" @@ -146,6 +202,69 @@ private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") {lastUpdated} } + + private def getAttemptURI(attemptInfo: ApplicationHistoryInfo, + returnEmptyIfAttemptInfoNull: Boolean = true ) = { + if (attemptInfo.appAttemptId.equals("")) { + if(returnEmptyIfAttemptInfoNull) { + attemptInfo.appAttemptId + } else { + HistoryServer.UI_PATH_PREFIX + s"/${attemptInfo.id}" + } + } else { + HistoryServer.UI_PATH_PREFIX + s"/${attemptInfo.id}" + "_" + s"${attemptInfo.appAttemptId}" + } + } + + private def firstAttemptRow(attemptInfo : ApplicationHistoryInfo) = { + val uiAddress = + if (attemptInfo.appAttemptId.equals("")) { + attemptInfo.appAttemptId + } else { + HistoryServer.UI_PATH_PREFIX + s"/${attemptInfo.id}" + "_" + s"${attemptInfo.appAttemptId}" + } + + val startTime = UIUtils.formatDate(attemptInfo.startTime) + val endTime = UIUtils.formatDate(attemptInfo.endTime) + val duration = UIUtils.formatDuration(attemptInfo.endTime - attemptInfo.startTime) + val lastUpdated = UIUtils.formatDate(attemptInfo.lastUpdated) + val attemptId = attemptInfo.appAttemptId + {attemptId} + {startTime} + {endTime} + + {duration} + {attemptInfo.sparkUser} + {lastUpdated} + } + + private def attemptRow(attemptInfo: ApplicationHistoryInfo) = { + + {firstAttemptRow(attemptInfo)} + + } + + private def appWithAttemptRow( + appAttemptsInfo: (String,ArrayBuffer[ApplicationHistoryInfo])): Seq[Node] = { + val applicationId = appAttemptsInfo._1 + val info = appAttemptsInfo._2 + val rowSpan = info.length + val rowSpanString = rowSpan.toString + val applicatioName = info(0).name + val lastAttemptURI = getAttemptURI(info(0), false) + val ttAttempts = info.slice(1, rowSpan -1) + val x = new xml.NodeBuffer + x += + + {applicationId} + {applicatioName} + { firstAttemptRow(info(0)) } + ; + for( i <- 1 until rowSpan ){ + x += attemptRow(info(i)) + } + x + } private def makePageLink(linkPage: Int, showIncomplete: Boolean): String = { "/?" + Array( diff --git a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala index 6d39a5e3fa64c..a591c7e046d5d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala @@ -26,6 +26,7 @@ package org.apache.spark.scheduler private[spark] class ApplicationEventListener extends SparkListener { var appName: Option[String] = None var appId: Option[String] = None + var appAttemptId: Option[String] = None var sparkUser: Option[String] = None var startTime: Option[Long] = None var endTime: Option[Long] = None @@ -35,6 +36,7 @@ private[spark] class ApplicationEventListener extends SparkListener { override def onApplicationStart(applicationStart: SparkListenerApplicationStart) { appName = Some(applicationStart.appName) appId = applicationStart.appId + appAttemptId = Some(applicationStart.appAttemptId) startTime = Some(applicationStart.time) sparkUser = Some(applicationStart.sparkUser) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 02a2263940821..49826a4ae2377 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -266,12 +266,11 @@ private[spark] object EventLoggingListener extends Logging { appAttemptId: String, compressionCodecName: Option[String] = None): String = { val name = appId.replaceAll("[ :/]", "-").replaceAll("[${}'\"]", "_").toLowerCase - - if (appAttemptId.equals("")) { - Utils.resolveURI(logBaseDir) + "/" + name.stripSuffix("/") - } else { - Utils.resolveURI(logBaseDir) + "/" + appAttemptId + "/" + name.stripSuffix("/") - } + if (appAttemptId.equals("")) { + logBaseDir.toString.stripSuffix("/") + "/" + name.stripSuffix("/") + } else { + logBaseDir.toString.stripSuffix("/") + "/" + name.stripSuffix("/") + "_" + appAttemptId + } } def getLogPath( diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index b711ff209af94..9f526b08a7b42 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -110,8 +110,8 @@ case class SparkListenerExecutorMetricsUpdate( extends SparkListenerEvent @DeveloperApi -case class SparkListenerApplicationStart(appName: String, appId: Option[String], time: Long, - sparkUser: String) extends SparkListenerEvent +case class SparkListenerApplicationStart(appName: String, appId: Option[String], + time: Long, sparkUser: String, appAttemptId: String = "") extends SparkListenerEvent @DeveloperApi case class SparkListenerApplicationEnd(time: Long) extends SparkListenerEvent diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 474f79fb756f6..9dbf466278eb0 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -194,7 +194,8 @@ private[spark] object JsonProtocol { ("App Name" -> applicationStart.appName) ~ ("App ID" -> applicationStart.appId.map(JString(_)).getOrElse(JNothing)) ~ ("Timestamp" -> applicationStart.time) ~ - ("User" -> applicationStart.sparkUser) + ("User" -> applicationStart.sparkUser) ~ + ("appAttemptId" -> applicationStart.appAttemptId) } def applicationEndToJson(applicationEnd: SparkListenerApplicationEnd): JValue = { @@ -562,7 +563,8 @@ private[spark] object JsonProtocol { val appId = Utils.jsonOption(json \ "App ID").map(_.extract[String]) val time = (json \ "Timestamp").extract[Long] val sparkUser = (json \ "User").extract[String] - SparkListenerApplicationStart(appName, appId, time, sparkUser) + val appAttemptId = (json \ "appAttemptId").extract[String] + SparkListenerApplicationStart(appName, appId, time, sparkUser, appAttemptId) } def applicationEndFromJson(json: JValue): SparkListenerApplicationEnd = { From 5fd5c6f44a56d2826e134cd03edcf5711558ecc6 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 7 Apr 2015 16:59:31 -0700 Subject: [PATCH 05/22] Fix my broken rebase. --- .../apache/spark/deploy/master/Master.scala | 20 +++++++++---------- .../scheduler/EventLoggingListener.scala | 11 ++-------- 2 files changed, 12 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 9a5d5877da86d..4994004180942 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -62,7 +62,7 @@ private[master] class Master( private val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs - + private val WORKER_TIMEOUT = conf.getLong("spark.worker.timeout", 60) * 1000 private val RETAINED_APPLICATIONS = conf.getInt("spark.deploy.retainedApplications", 200) private val RETAINED_DRIVERS = conf.getInt("spark.deploy.retainedDrivers", 200) @@ -86,7 +86,7 @@ private[master] class Master( private val drivers = new HashSet[DriverInfo] private val completedDrivers = new ArrayBuffer[DriverInfo] // Drivers currently spooled for scheduling - private val waitingDrivers = new ArrayBuffer[DriverInfo] + private val waitingDrivers = new ArrayBuffer[DriverInfo] private var nextDriverNumber = 0 Utils.checkHost(host, "Expected hostname") @@ -753,24 +753,24 @@ private[master] class Master( app.desc.appUiUrl = notFoundBasePath return false } - + val eventLogFilePrefix = EventLoggingListener.getLogPath( - eventLogDir, app.id, app.desc.eventLogCodec) + eventLogDir, app.id, "", app.desc.eventLogCodec) val fs = Utils.getHadoopFileSystem(eventLogDir, hadoopConf) - val inProgressExists = fs.exists(new Path(eventLogFilePrefix + + val inProgressExists = fs.exists(new Path(eventLogFilePrefix + EventLoggingListener.IN_PROGRESS)) - + if (inProgressExists) { // Event logging is enabled for this application, but the application is still in progress logWarning(s"Application $appName is still in progress, it may be terminated abnormally.") } - + val (eventLogFile, status) = if (inProgressExists) { (eventLogFilePrefix + EventLoggingListener.IN_PROGRESS, " (in progress)") } else { (eventLogFilePrefix, " (completed)") } - + val logInput = EventLoggingListener.openEventLog(new Path(eventLogFile), fs) val replayBus = new ReplayListenerBus() val ui = SparkUI.createHistoryUI(new SparkConf, replayBus, new SecurityManager(conf), @@ -854,8 +854,8 @@ private[master] class Master( } private def removeDriver( - driverId: String, - finalState: DriverState, + driverId: String, + finalState: DriverState, exception: Option[Exception]) { drivers.find(d => d.id == driverId) match { case Some(driver) => diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 49826a4ae2377..c75c7a53afefa 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -90,7 +90,7 @@ private[spark] class EventLoggingListener( private[scheduler] val loggedEvents = new ArrayBuffer[JValue] // Visible for tests only. - private[scheduler] val logPath = getLogPath(logBaseDir, appId, compressionCodecName, appAttemptId) + private[scheduler] val logPath = getLogPath(logBaseDir, appId, appAttemptId, compressionCodecName) /** * Creates the log file in the configured log directory. @@ -261,7 +261,7 @@ private[spark] object EventLoggingListener extends Logging { * @return A path which consists of file-system-safe characters. */ def getLogPath( - logBaseDir: String, + logBaseDir: URI, appId: String, appAttemptId: String, compressionCodecName: Option[String] = None): String = { @@ -273,13 +273,6 @@ private[spark] object EventLoggingListener extends Logging { } } - def getLogPath( - logBaseDir: String, - appId: String, - compressionCodecName: Option[String] = None): String = { - getLogPath(logBaseDir, appId, "", compressionCodecName) - } - /** * Opens an event log file and returns an input stream that contains the event data. * From 3245aa254bb2c9f2152db1d3be206c0d8e1f851d Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 8 Apr 2015 11:24:41 -0700 Subject: [PATCH 06/22] Make app attempts part of the history server model. This change explicitly models app attempts in the history server. An app is now a collection of app attempts, instead of the logic to match apps to app attempts being implicit in the rendering code. This makes the rendering code a lot simpler, since it doesn't need to do any fancy processing of the app list to figure out what to show. --- .../history/ApplicationHistoryProvider.scala | 15 +- .../deploy/history/FsHistoryProvider.scala | 151 ++++++++------ .../spark/deploy/history/HistoryPage.scala | 192 +++++++----------- .../spark/deploy/history/HistoryServer.scala | 6 +- .../history/FsHistoryProviderSuite.scala | 31 +-- .../deploy/history/HistoryServerSuite.scala | 3 +- .../scheduler/EventLoggingListenerSuite.scala | 12 +- 7 files changed, 199 insertions(+), 211 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala index 7509b6d6b9cf7..166afc97a6c12 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala @@ -19,15 +19,18 @@ package org.apache.spark.deploy.history import org.apache.spark.ui.SparkUI -private[history] case class ApplicationHistoryInfo( - id: String, +private[history] case class ApplicationAttemptInfo( + attemptId: String, name: String, startTime: Long, endTime: Long, lastUpdated: Long, sparkUser: String, - completed: Boolean = false, - appAttemptId: String = "") + completed: Boolean = false) + +private[history] case class ApplicationHistoryInfo( + id: String, + attempts: List[ApplicationAttemptInfo]) private[history] abstract class ApplicationHistoryProvider { @@ -42,9 +45,11 @@ private[history] abstract class ApplicationHistoryProvider { * Returns the Spark UI for a specific application. * * @param appId The application ID. + * @param attemptId The application attempt ID for apps with multiple attempts (or an empty + * string for apps with a single attempt). * @return The application's UI, or None if application is not found. */ - def getAppUI(appId: String): Option[SparkUI] + def getAppUI(appId: String, attemptId: String): Option[SparkUI] /** * Called when the server is shutting down. diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 7e31a299ae925..2f48a20f1e5a6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -143,31 +143,34 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis override def getListing(): Iterable[FsApplicationHistoryInfo] = applications.values - override def getAppUI(appId: String): Option[SparkUI] = { + override def getAppUI(appId: String, attemptId: String): Option[SparkUI] = { try { - applications.get(appId).map { info => - val replayBus = new ReplayListenerBus() - val ui = { - val conf = this.conf.clone() - val appSecManager = new SecurityManager(conf) - SparkUI.createHistoryUI(conf, replayBus, appSecManager, appId, - s"${HistoryServer.UI_PATH_PREFIX}/$appId") - // Do not call ui.bind() to avoid creating a new server for each application - } + applications.get(appId).flatMap { info => + val attempts = info.attempts.filter(_.attemptId == attemptId) + attempts.headOption.map { attempt => + val replayBus = new ReplayListenerBus() + val ui = { + val conf = this.conf.clone() + val appSecManager = new SecurityManager(conf) + SparkUI.createHistoryUI(conf, replayBus, appSecManager, appId, + s"${HistoryServer.UI_PATH_PREFIX}/$appId") + // Do not call ui.bind() to avoid creating a new server for each application + } - val appListener = new ApplicationEventListener() - replayBus.addListener(appListener) - val appInfo = replay(fs.getFileStatus(new Path(logDir, info.logPath)), replayBus) + val appListener = new ApplicationEventListener() + replayBus.addListener(appListener) + val appInfo = replay(fs.getFileStatus(new Path(logDir, attempt.logPath)), replayBus) - ui.setAppName(s"${appInfo.name} ($appId)") + ui.setAppName(s"${attempt.name} ($appId)") - val uiAclsEnabled = conf.getBoolean("spark.history.ui.acls.enable", false) - ui.getSecurityManager.setAcls(uiAclsEnabled) - // make sure to set admin acls before view acls so they are properly picked up - ui.getSecurityManager.setAdminAcls(appListener.adminAcls.getOrElse("")) - ui.getSecurityManager.setViewAcls(appInfo.sparkUser, - appListener.viewAcls.getOrElse("")) - ui + val uiAclsEnabled = conf.getBoolean("spark.history.ui.acls.enable", false) + ui.getSecurityManager.setAcls(uiAclsEnabled) + // make sure to set admin acls before view acls so they are properly picked up + ui.getSecurityManager.setAdminAcls(appListener.adminAcls.getOrElse("")) + ui.getSecurityManager.setViewAcls(attempt.sparkUser, + appListener.viewAcls.getOrElse("")) + ui + } } } catch { case e: FileNotFoundException => None @@ -225,7 +228,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis */ private def mergeApplicationListing(logs: Seq[FileStatus]): Unit = { val bus = new ReplayListenerBus() - val newApps = logs.flatMap { fileStatus => + val newAttempts = logs.flatMap { fileStatus => try { val res = replay(fileStatus, bus) logInfo(s"Application log ${res.logPath} loaded successfully.") @@ -237,41 +240,52 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis e) None } - }.toSeq.sortWith(compareAppInfo) - - // When there are new logs, merge the new list with the existing one, maintaining - // the expected ordering (descending end time). Maintaining the order is important - // to avoid having to sort the list every time there is a request for the log list. - if (newApps.nonEmpty) { - val mergedApps = new mutable.LinkedHashMap[String, FsApplicationHistoryInfo]() - def addIfAbsent(info: FsApplicationHistoryInfo): Unit = { - if (!mergedApps.contains(info.id) || - mergedApps(info.id).logPath.endsWith(EventLoggingListener.IN_PROGRESS) && - !info.logPath.endsWith(EventLoggingListener.IN_PROGRESS)) { - val key = - if (info.appAttemptId.equals("")) { - info.id - } else { - info.id + "_" + info.appAttemptId - } - mergedApps += (key -> info) - } - } + } - val newIterator = newApps.iterator.buffered - val oldIterator = applications.values.iterator.buffered - while (newIterator.hasNext && oldIterator.hasNext) { - if (compareAppInfo(newIterator.head, oldIterator.head)) { - addIfAbsent(newIterator.next()) - } else { - addIfAbsent(oldIterator.next()) + if (newAttempts.isEmpty) { + return + } + + // Build a map containing all apps that contain new attempts. The app information in this map + // contains both the new app attempt, and those that were already loaded in the existing apps + // map. If an attempt has been updated, it replaces the old attempt in the list. + val newAppMap = new mutable.HashMap[String, FsApplicationHistoryInfo]() + newAttempts.foreach { attempt => + val appInfo = applications.get(attempt.appId) + .orElse(newAppMap.get(attempt.appId)) + .map { app => + val attempts = + app.attempts.filter(_.attemptId != attempt.attemptId).toList ++ List(attempt) + new FsApplicationHistoryInfo(attempt.appId, attempts.sortWith(compareAttemptInfo)) } + .getOrElse(new FsApplicationHistoryInfo(attempt.appId, List(attempt))) + newAppMap(attempt.appId) = appInfo + } + + // Merge the new app list with the existing one, maintaining the expected ordering (descending + // end time). Maintaining the order is important to avoid having to sort the list every time + // there is a request for the log list. + val newApps = newAppMap.values.toSeq.sortWith(compareAppInfo) + val mergedApps = new mutable.LinkedHashMap[String, FsApplicationHistoryInfo]() + def addIfAbsent(info: FsApplicationHistoryInfo): Unit = { + if (!mergedApps.contains(info.id)) { + mergedApps += (info.id -> info) } - newIterator.foreach(addIfAbsent) - oldIterator.foreach(addIfAbsent) + } - applications = mergedApps + val newIterator = newApps.iterator.buffered + val oldIterator = applications.values.iterator.buffered + while (newIterator.hasNext && oldIterator.hasNext) { + if (compareAppInfo(newIterator.head, oldIterator.head)) { + addIfAbsent(newIterator.next()) + } else { + addIfAbsent(oldIterator.next()) + } } + newIterator.foreach(addIfAbsent) + oldIterator.foreach(addIfAbsent) + + applications = mergedApps } /** @@ -288,7 +302,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis val appsToRetain = new mutable.LinkedHashMap[String, FsApplicationHistoryInfo]() applications.values.foreach { info => - if (now - info.lastUpdated <= maxAge) { + if (now - info.attempts.head.lastUpdated <= maxAge) { appsToRetain += (info.id -> info) } } @@ -321,6 +335,12 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis private def compareAppInfo( i1: FsApplicationHistoryInfo, i2: FsApplicationHistoryInfo): Boolean = { + compareAttemptInfo(i1.attempts.head, i2.attempts.head) + } + + private def compareAttemptInfo( + i1: FsApplicationAttemptInfo, + i2: FsApplicationAttemptInfo): Boolean = { if (i1.endTime != i2.endTime) i1.endTime >= i2.endTime else i1.startTime >= i2.startTime } @@ -328,7 +348,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis * Replays the events in the specified log file and returns information about the associated * application. */ - private def replay(eventLog: FileStatus, bus: ReplayListenerBus): FsApplicationHistoryInfo = { + private def replay(eventLog: FileStatus, bus: ReplayListenerBus): FsApplicationAttemptInfo = { val logPath = eventLog.getPath() logInfo(s"Replaying log path: $logPath") val logInput = @@ -341,16 +361,16 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis val appListener = new ApplicationEventListener bus.addListener(appListener) bus.replay(logInput, logPath.toString) - new FsApplicationHistoryInfo( + new FsApplicationAttemptInfo( logPath.getName(), appListener.appId.getOrElse(logPath.getName()), + appListener.appAttemptId.getOrElse(""), appListener.appName.getOrElse(NOT_STARTED), appListener.startTime.getOrElse(-1L), appListener.endTime.getOrElse(-1L), getModificationTime(eventLog).get, appListener.sparkUser.getOrElse(NOT_STARTED), - isApplicationCompleted(eventLog), - appListener.appAttemptId.getOrElse("")) + isApplicationCompleted(eventLog)) } finally { logInput.close() } @@ -437,15 +457,20 @@ private object FsHistoryProvider { val DEFAULT_SPARK_HISTORY_FS_MAXAGE_S = Duration(7, TimeUnit.DAYS).toSeconds } -private class FsApplicationHistoryInfo( +private class FsApplicationAttemptInfo( val logPath: String, - id: String, + val appId: String, + attemptId: String, name: String, startTime: Long, endTime: Long, lastUpdated: Long, sparkUser: String, - completed: Boolean = true, - appAttemptId: String ="") - extends ApplicationHistoryInfo( - id, name, startTime, endTime, lastUpdated, sparkUser, completed, appAttemptId) + completed: Boolean = true) + extends ApplicationAttemptInfo( + attemptId, name, startTime, endTime, lastUpdated, sparkUser, completed) + +private class FsApplicationHistoryInfo( + id: String, + override val attempts: List[FsApplicationAttemptInfo]) + extends ApplicationHistoryInfo(id, attempts) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index 87159715b34e8..e51256c40122b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -37,18 +37,13 @@ private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") val requestedIncomplete = Option(request.getParameter("showIncomplete")).getOrElse("false").toBoolean - val allCompletedAppsNAttempts = - parent.getApplicationList().filter(_.completed != requestedIncomplete) - val (hasAttemptInfo, appToAttemptMap) = getApplicationLevelList(allCompletedAppsNAttempts) - - val allAppsSize = allCompletedAppsNAttempts.size - + val allApps = parent.getApplicationList() + .filter(_.attempts.exists(_.completed != requestedIncomplete)) + val allAppsSize = allApps.size + val actualFirst = if (requestedFirst < allAppsSize) requestedFirst else 0 - val apps = - allCompletedAppsNAttempts.slice(actualFirst, Math.min(actualFirst + pageSize, allAppsSize)) - val appWithAttemptsDisplayList = - appToAttemptMap.slice(actualFirst, Math.min(actualFirst + pageSize, allAppsSize)) - + val appsToShow = allApps.slice(actualFirst, Math.min(actualFirst + pageSize, allAppsSize)) + val actualPage = (actualFirst / pageSize) + 1 val last = Math.min(actualFirst + pageSize, allAppsSize) - 1 val pageCount = allAppsSize / pageSize + (if (allAppsSize % pageSize > 0) 1 else 0) @@ -56,12 +51,14 @@ private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") val secondPageFromLeft = 2 val secondPageFromRight = pageCount - 1 - val appTable = - if (hasAttemptInfo) { - UIUtils.listingTable(appWithAttemptHeader, appWithAttemptRow, appWithAttemptsDisplayList) + val hasMultipleAttempts = appsToShow.exists(_.attempts.size > 1) + val appTable = + if (hasMultipleAttempts) { + UIUtils.listingTable(appWithAttemptHeader, appWithAttemptRow, appsToShow) } else { - UIUtils.listingTable(appHeader, appRow, apps) + UIUtils.listingTable(appHeader, appRow, appsToShow) } + val providerConfig = parent.getProviderConfig() val content =
@@ -129,36 +126,6 @@ private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("")
UIUtils.basicSparkPage(content, "History Server") } - - private def getApplicationLevelList (appNattemptList: Iterable[ApplicationHistoryInfo]) ={ - // Create HashMap as per the multiple attempts for one application. - // If there is no attempt specific stuff, then - // do return false, to indicate the same, so that previous UI gets displayed. - var hasAttemptInfo = false - val appToAttemptInfo = new HashMap[String, ArrayBuffer[ApplicationHistoryInfo]] - for( appAttempt <- appNattemptList) { - if(!appAttempt.appAttemptId.equals("")){ - hasAttemptInfo = true - val attemptId = appAttempt.appAttemptId.toInt - if(appToAttemptInfo.contains(appAttempt.id)){ - val currentAttempts = appToAttemptInfo.get(appAttempt.id).get - currentAttempts += appAttempt - appToAttemptInfo.put( appAttempt.id, currentAttempts) - } else { - val currentAttempts = new ArrayBuffer[ApplicationHistoryInfo]() - currentAttempts += appAttempt - appToAttemptInfo.put( appAttempt.id, currentAttempts ) - } - }else { - val currentAttempts = new ArrayBuffer[ApplicationHistoryInfo]() - currentAttempts += appAttempt - appToAttemptInfo.put(appAttempt.id, currentAttempts) - } - } - val sortedMap = ListMap(appToAttemptInfo.toSeq.sortWith(_._1 > _._1):_*) - (hasAttemptInfo, sortedMap) - } - private val appHeader = Seq( "App ID", @@ -169,12 +136,6 @@ private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") "Spark User", "Last Updated") - private def rangeIndices(range: Seq[Int], condition: Int => Boolean, showIncomplete: Boolean): - Seq[Node] = { - range.filter(condition).map(nextPage => - {nextPage} ) - } - private val appWithAttemptHeader = Seq( "App ID", "App Name", @@ -185,85 +146,70 @@ private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") "Spark User", "Last Updated") - private def appRow(info: ApplicationHistoryInfo): Seq[Node] = { - val uiAddress = HistoryServer.UI_PATH_PREFIX + s"/${info.id}" - val startTime = UIUtils.formatDate(info.startTime) - val endTime = if (info.endTime > 0) UIUtils.formatDate(info.endTime) else "-" - val duration = - if (info.endTime > 0) UIUtils.formatDuration(info.endTime - info.startTime) else "-" - val lastUpdated = UIUtils.formatDate(info.lastUpdated) - - {info.id} - {info.name} - {startTime} - {endTime} - {duration} - {info.sparkUser} - {lastUpdated} - + private def rangeIndices( + range: Seq[Int], + condition: Int => Boolean, + showIncomplete: Boolean): Seq[Node] = { + range.filter(condition).map(nextPage => + {nextPage} ) } - - private def getAttemptURI(attemptInfo: ApplicationHistoryInfo, - returnEmptyIfAttemptInfoNull: Boolean = true ) = { - if (attemptInfo.appAttemptId.equals("")) { - if(returnEmptyIfAttemptInfoNull) { - attemptInfo.appAttemptId - } else { - HistoryServer.UI_PATH_PREFIX + s"/${attemptInfo.id}" - } - } else { - HistoryServer.UI_PATH_PREFIX + s"/${attemptInfo.id}" + "_" + s"${attemptInfo.appAttemptId}" - } + + private def getAttemptURI(appId: String, attemptInfo: ApplicationAttemptInfo): String = { + val attemptSuffix = if (!attemptInfo.attemptId.isEmpty) s"/${attemptInfo.attemptId}" else "" + s"${HistoryServer.UI_PATH_PREFIX}/${appId}${attemptSuffix}" } - - private def firstAttemptRow(attemptInfo : ApplicationHistoryInfo) = { - val uiAddress = - if (attemptInfo.appAttemptId.equals("")) { - attemptInfo.appAttemptId - } else { - HistoryServer.UI_PATH_PREFIX + s"/${attemptInfo.id}" + "_" + s"${attemptInfo.appAttemptId}" - } - + + private def attemptRow( + info: ApplicationHistoryInfo, + attempt: ApplicationAttemptInfo, + isFirst: Boolean): Seq[Node] = { + val attemptInfo = info.attempts.head + val uiAddress = HistoryServer.UI_PATH_PREFIX + s"/${info.id}" val startTime = UIUtils.formatDate(attemptInfo.startTime) - val endTime = UIUtils.formatDate(attemptInfo.endTime) - val duration = UIUtils.formatDuration(attemptInfo.endTime - attemptInfo.startTime) + val endTime = if (attemptInfo.endTime > 0) UIUtils.formatDate(attemptInfo.endTime) else "-" + val duration = + if (attemptInfo.endTime > 0) { + UIUtils.formatDuration(attemptInfo.endTime - attemptInfo.startTime) + } else { + "-" + } val lastUpdated = UIUtils.formatDate(attemptInfo.lastUpdated) - val attemptId = attemptInfo.appAttemptId - {attemptId} - {startTime} - {endTime} - - {duration} - {attemptInfo.sparkUser} - {lastUpdated} - } - - private def attemptRow(attemptInfo: ApplicationHistoryInfo) = { - {firstAttemptRow(attemptInfo)} + { + if (isFirst) { + if (info.attempts.size > 1) { + {info.id} + } else { + {info.id} + } + } else { + new xml.Comment("") + } + } + { + if (info.attempts.size > 1 && !attempt.attemptId.isEmpty) { + {attempt.attemptId} + } else { + Nil + } + } + {attempt.name} + {startTime} + {endTime} + + {duration} + {attempt.sparkUser} + {lastUpdated} } - - private def appWithAttemptRow( - appAttemptsInfo: (String,ArrayBuffer[ApplicationHistoryInfo])): Seq[Node] = { - val applicationId = appAttemptsInfo._1 - val info = appAttemptsInfo._2 - val rowSpan = info.length - val rowSpanString = rowSpan.toString - val applicatioName = info(0).name - val lastAttemptURI = getAttemptURI(info(0), false) - val ttAttempts = info.slice(1, rowSpan -1) - val x = new xml.NodeBuffer - x += - - {applicationId} - {applicatioName} - { firstAttemptRow(info(0)) } - ; - for( i <- 1 until rowSpan ){ - x += attemptRow(info(i)) - } - x + + private def appRow(info: ApplicationHistoryInfo): Seq[Node] = { + attemptRow(info, info.attempts.head, true) + } + + private def appWithAttemptRow(info: ApplicationHistoryInfo): Seq[Node] = { + attemptRow(info, info.attempts.head, true) ++ + info.attempts.drop(1).flatMap(attemptRow(info, _, false)) } private def makePageLink(linkPage: Int, showIncomplete: Boolean): String = { diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 72f6048239297..b7b4355351619 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -52,7 +52,11 @@ class HistoryServer( private val appLoader = new CacheLoader[String, SparkUI] { override def load(key: String): SparkUI = { - val ui = provider.getAppUI(key).getOrElse(throw new NoSuchElementException()) + val parts = key.split("/") + require(parts.length == 1 || parts.length == 2, s"Invalid app key $key") + val ui = provider + .getAppUI(parts(0), if (parts.length > 1) parts(1) else "") + .getOrElse(throw new NoSuchElementException()) attachSparkUI(ui) ui } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index fcae603c7d18e..a75ade3cc6f18 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -50,7 +50,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers inProgress: Boolean, codec: Option[String] = None): File = { val ip = if (inProgress) EventLoggingListener.IN_PROGRESS else "" - val logUri = EventLoggingListener.getLogPath(testDir.toURI, appId) + val logUri = EventLoggingListener.getLogPath(testDir.toURI, appId, "") val logPath = new URI(logUri).getPath + ip new File(logPath) } @@ -106,22 +106,28 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers val list = provider.getListing().toSeq list should not be (null) list.size should be (5) - list.count(_.completed) should be (3) + list.count(_.attempts.head.completed) should be (3) - list(0) should be (ApplicationHistoryInfo(newAppComplete.getName(), "new-app-complete", 1L, 5L, + def makeAppInfo(id: String, name: String, start: Long, end: Long, lastMod: Long, + user: String, completed: Boolean): ApplicationHistoryInfo = { + ApplicationHistoryInfo(id, + List(ApplicationAttemptInfo("", name, start, end, lastMod, user, completed))) + } + + list(0) should be (makeAppInfo(newAppComplete.getName(), "new-app-complete", 1L, 5L, newAppComplete.lastModified(), "test", true)) - list(1) should be (ApplicationHistoryInfo(newAppCompressedComplete.getName(), + list(1) should be (makeAppInfo(newAppCompressedComplete.getName(), "new-app-compressed-complete", 1L, 4L, newAppCompressedComplete.lastModified(), "test", true)) - list(2) should be (ApplicationHistoryInfo(oldAppComplete.getName(), "old-app-complete", 2L, 3L, + list(2) should be (makeAppInfo(oldAppComplete.getName(), "old-app-complete", 2L, 3L, oldAppComplete.lastModified(), "test", true)) - list(3) should be (ApplicationHistoryInfo(oldAppIncomplete.getName(), "old-app-incomplete", 2L, - -1L, oldAppIncomplete.lastModified(), "test", false)) - list(4) should be (ApplicationHistoryInfo(newAppIncomplete.getName(), "new-app-incomplete", 1L, - -1L, newAppIncomplete.lastModified(), "test", false)) + list(3) should be (makeAppInfo(oldAppIncomplete.getName(), "old-app-incomplete", 2L, -1L, + oldAppIncomplete.lastModified(), "test", false)) + list(4) should be (makeAppInfo(newAppIncomplete.getName(), "new-app-incomplete", 1L, -1L, + newAppIncomplete.lastModified(), "test", false)) // Make sure the UI can be rendered. list.foreach { case info => - val appUi = provider.getAppUI(info.id) + val appUi = provider.getAppUI(info.id, "") appUi should not be null } } @@ -190,13 +196,14 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers provider.checkForLogs() val appListBeforeRename = provider.getListing() appListBeforeRename.size should be (1) - appListBeforeRename.head.logPath should endWith(EventLoggingListener.IN_PROGRESS) + appListBeforeRename.head.attempts.head.logPath should endWith(EventLoggingListener.IN_PROGRESS) logFile1.renameTo(newLogFile("app1", inProgress = false)) provider.checkForLogs() val appListAfterRename = provider.getListing() appListAfterRename.size should be (1) - appListAfterRename.head.logPath should not endWith(EventLoggingListener.IN_PROGRESS) + appListAfterRename.head.attempts.head.logPath should not + endWith(EventLoggingListener.IN_PROGRESS) } test("SPARK-5582: empty log directory") { diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index 3a9963a5ce7b7..b750a9e49d1be 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -36,7 +36,8 @@ class HistoryServerSuite extends FunSuite with Matchers with MockitoSugar { val request = mock[HttpServletRequest] val ui = mock[SparkUI] val link = "/history/app1" - val info = new ApplicationHistoryInfo("app1", "app1", 0, 2, 1, "xxx", true) + val info = new ApplicationHistoryInfo("app1", + List(ApplicationAttemptInfo("attempt1", "app1", 0, 2, 1, "xxx", true))) when(historyServer.getApplicationList()).thenReturn(Seq(info)) when(ui.basePath).thenReturn(link) when(historyServer.getProviderConfig()).thenReturn(Map[String, String]()) diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index 30ee63e78d9d8..2c3efd8b71e2a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -95,7 +95,7 @@ class EventLoggingListenerSuite extends FunSuite with LocalSparkContext with Bef } test("Log overwriting") { - val logUri = EventLoggingListener.getLogPath(testDir.toURI, "test") + val logUri = EventLoggingListener.getLogPath(testDir.toURI, "test", "") val logPath = new URI(logUri).getPath // Create file before writing the event log new FileOutputStream(new File(logPath)).close() @@ -108,18 +108,18 @@ class EventLoggingListenerSuite extends FunSuite with LocalSparkContext with Bef test("Event log name") { // without compression assert(s"file:/base-dir/app1" === EventLoggingListener.getLogPath( - Utils.resolveURI("/base-dir"), "app1")) + Utils.resolveURI("/base-dir"), "app1", "")) // with compression assert(s"file:/base-dir/app1.lzf" === - EventLoggingListener.getLogPath(Utils.resolveURI("/base-dir"), "app1", Some("lzf"))) + EventLoggingListener.getLogPath(Utils.resolveURI("/base-dir"), "app1", "", Some("lzf"))) // illegal characters in app ID assert(s"file:/base-dir/a-fine-mind_dollar_bills__1" === EventLoggingListener.getLogPath(Utils.resolveURI("/base-dir"), - "a fine:mind$dollar{bills}.1")) + "a fine:mind$dollar{bills}.1", "")) // illegal characters in app ID with compression assert(s"file:/base-dir/a-fine-mind_dollar_bills__1.lz4" === EventLoggingListener.getLogPath(Utils.resolveURI("/base-dir"), - "a fine:mind$dollar{bills}.1", Some("lz4"))) + "a fine:mind$dollar{bills}.1", "", Some("lz4"))) } /* ----------------- * @@ -186,7 +186,7 @@ class EventLoggingListenerSuite extends FunSuite with LocalSparkContext with Bef val eventLogPath = eventLogger.logPath val expectedLogDir = testDir.toURI() assert(eventLogPath === EventLoggingListener.getLogPath( - expectedLogDir, sc.applicationId, compressionCodec.map(CompressionCodec.getShortName))) + expectedLogDir, sc.applicationId, "", compressionCodec.map(CompressionCodec.getShortName))) // Begin listening for events that trigger asserts val eventExistenceListener = new EventExistenceListener(eventLogger) From 88b1de8d9ebfd3cef78161c9cbc4d3a06639a933 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 8 Apr 2015 14:45:29 -0700 Subject: [PATCH 07/22] Add a test for apps with multiple attempts. --- .../deploy/history/FsHistoryProvider.scala | 4 +- .../history/FsHistoryProviderSuite.scala | 174 +++++++++++++----- 2 files changed, 126 insertions(+), 52 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 2f48a20f1e5a6..2798ec0237619 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -251,8 +251,8 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis // map. If an attempt has been updated, it replaces the old attempt in the list. val newAppMap = new mutable.HashMap[String, FsApplicationHistoryInfo]() newAttempts.foreach { attempt => - val appInfo = applications.get(attempt.appId) - .orElse(newAppMap.get(attempt.appId)) + val appInfo = newAppMap.get(attempt.appId) + .orElse(applications.get(attempt.appId)) .map { app => val attempts = app.attempts.filter(_.attemptId != attempt.attemptId).toList ++ List(attempt) diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index a75ade3cc6f18..78ab93247d8c7 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -47,10 +47,11 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers /** Create a fake log file using the new log format used in Spark 1.3+ */ private def newLogFile( appId: String, + appAttemptId: String, inProgress: Boolean, codec: Option[String] = None): File = { val ip = if (inProgress) EventLoggingListener.IN_PROGRESS else "" - val logUri = EventLoggingListener.getLogPath(testDir.toURI, appId, "") + val logUri = EventLoggingListener.getLogPath(testDir.toURI, appId, appAttemptId) val logPath = new URI(logUri).getPath + ip new File(logPath) } @@ -59,20 +60,20 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers val provider = new FsHistoryProvider(createTestConf()) // Write a new-style application log. - val newAppComplete = newLogFile("new1", inProgress = false) + val newAppComplete = newLogFile("new1", "", inProgress = false) writeFile(newAppComplete, true, None, SparkListenerApplicationStart("new-app-complete", None, 1L, "test"), SparkListenerApplicationEnd(5L) ) // Write a new-style application log. - val newAppCompressedComplete = newLogFile("new1compressed", inProgress = false, Some("lzf")) + val newAppCompressedComplete = newLogFile("new1compressed", "", inProgress = false, Some("lzf")) writeFile(newAppCompressedComplete, true, None, SparkListenerApplicationStart("new-app-compressed-complete", None, 1L, "test"), SparkListenerApplicationEnd(4L)) // Write an unfinished app, new-style. - val newAppIncomplete = newLogFile("new2", inProgress = true) + val newAppIncomplete = newLogFile("new2", "", inProgress = true) writeFile(newAppIncomplete, true, None, SparkListenerApplicationStart("new-app-incomplete", None, 1L, "test") ) @@ -101,34 +102,33 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers // Force a reload of data from the log directory, and check that both logs are loaded. // Take the opportunity to check that the offset checks work as expected. - provider.checkForLogs() - - val list = provider.getListing().toSeq - list should not be (null) - list.size should be (5) - list.count(_.attempts.head.completed) should be (3) - - def makeAppInfo(id: String, name: String, start: Long, end: Long, lastMod: Long, - user: String, completed: Boolean): ApplicationHistoryInfo = { - ApplicationHistoryInfo(id, - List(ApplicationAttemptInfo("", name, start, end, lastMod, user, completed))) - } + updateAndCheck(provider) { list => + list.size should be (5) + list.count(_.attempts.head.completed) should be (3) + + def makeAppInfo(id: String, name: String, start: Long, end: Long, lastMod: Long, + user: String, completed: Boolean): ApplicationHistoryInfo = { + ApplicationHistoryInfo(id, + List(ApplicationAttemptInfo("", name, start, end, lastMod, user, completed))) + } - list(0) should be (makeAppInfo(newAppComplete.getName(), "new-app-complete", 1L, 5L, - newAppComplete.lastModified(), "test", true)) - list(1) should be (makeAppInfo(newAppCompressedComplete.getName(), - "new-app-compressed-complete", 1L, 4L, newAppCompressedComplete.lastModified(), "test", true)) - list(2) should be (makeAppInfo(oldAppComplete.getName(), "old-app-complete", 2L, 3L, - oldAppComplete.lastModified(), "test", true)) - list(3) should be (makeAppInfo(oldAppIncomplete.getName(), "old-app-incomplete", 2L, -1L, - oldAppIncomplete.lastModified(), "test", false)) - list(4) should be (makeAppInfo(newAppIncomplete.getName(), "new-app-incomplete", 1L, -1L, - newAppIncomplete.lastModified(), "test", false)) - - // Make sure the UI can be rendered. - list.foreach { case info => - val appUi = provider.getAppUI(info.id, "") - appUi should not be null + list(0) should be (makeAppInfo(newAppComplete.getName(), "new-app-complete", 1L, 5L, + newAppComplete.lastModified(), "test", true)) + list(1) should be (makeAppInfo(newAppCompressedComplete.getName(), + "new-app-compressed-complete", 1L, 4L, newAppCompressedComplete.lastModified(), "test", + true)) + list(2) should be (makeAppInfo(oldAppComplete.getName(), "old-app-complete", 2L, 3L, + oldAppComplete.lastModified(), "test", true)) + list(3) should be (makeAppInfo(oldAppIncomplete.getName(), "old-app-incomplete", 2L, -1L, + oldAppIncomplete.lastModified(), "test", false)) + list(4) should be (makeAppInfo(newAppIncomplete.getName(), "new-app-incomplete", 1L, -1L, + newAppIncomplete.lastModified(), "test", false)) + + // Make sure the UI can be rendered. + list.foreach { case info => + val appUi = provider.getAppUI(info.id, "") + appUi should not be null + } } } @@ -165,12 +165,12 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers } test("SPARK-3697: ignore directories that cannot be read.") { - val logFile1 = newLogFile("new1", inProgress = false) + val logFile1 = newLogFile("new1", "", inProgress = false) writeFile(logFile1, true, None, SparkListenerApplicationStart("app1-1", None, 1L, "test"), SparkListenerApplicationEnd(2L) ) - val logFile2 = newLogFile("new2", inProgress = false) + val logFile2 = newLogFile("new2", "", inProgress = false) writeFile(logFile2, true, None, SparkListenerApplicationStart("app1-2", None, 1L, "test"), SparkListenerApplicationEnd(2L) @@ -178,38 +178,37 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers logFile2.setReadable(false, false) val provider = new FsHistoryProvider(createTestConf()) - provider.checkForLogs() - - val list = provider.getListing().toSeq - list should not be (null) - list.size should be (1) + updateAndCheck(provider) { list => + list.size should be (1) + } } test("history file is renamed from inprogress to completed") { val provider = new FsHistoryProvider(createTestConf()) - val logFile1 = newLogFile("app1", inProgress = true) + val logFile1 = newLogFile("app1", "", inProgress = true) writeFile(logFile1, true, None, SparkListenerApplicationStart("app1", Some("app1"), 1L, "test"), SparkListenerApplicationEnd(2L) ) - provider.checkForLogs() - val appListBeforeRename = provider.getListing() - appListBeforeRename.size should be (1) - appListBeforeRename.head.attempts.head.logPath should endWith(EventLoggingListener.IN_PROGRESS) + updateAndCheck(provider) { list => + list.size should be (1) + list.head.attempts.head.asInstanceOf[FsApplicationAttemptInfo].logPath should + endWith(EventLoggingListener.IN_PROGRESS) + } - logFile1.renameTo(newLogFile("app1", inProgress = false)) - provider.checkForLogs() - val appListAfterRename = provider.getListing() - appListAfterRename.size should be (1) - appListAfterRename.head.attempts.head.logPath should not - endWith(EventLoggingListener.IN_PROGRESS) + logFile1.renameTo(newLogFile("app1", "", inProgress = false)) + updateAndCheck(provider) { list => + list.size should be (1) + list.head.attempts.head.asInstanceOf[FsApplicationAttemptInfo].logPath should not + endWith(EventLoggingListener.IN_PROGRESS) + } } test("SPARK-5582: empty log directory") { val provider = new FsHistoryProvider(createTestConf()) - val logFile1 = newLogFile("app1", inProgress = true) + val logFile1 = newLogFile("app1", "", inProgress = true) writeFile(logFile1, true, None, SparkListenerApplicationStart("app1", Some("app1"), 1L, "test"), SparkListenerApplicationEnd(2L)) @@ -222,6 +221,81 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers appListAfterRename.size should be (1) } + test("apps with multiple attempts") { + val provider = new FsHistoryProvider(createTestConf()) + + val attempt1 = newLogFile("app1", "attempt1", inProgress = false) + writeFile(attempt1, true, None, + SparkListenerApplicationStart("app1", Some("app1"), 1L, "test", "attempt1"), + SparkListenerApplicationEnd(2L) + ) + + updateAndCheck(provider) { list => + list.size should be (1) + list.head.attempts.size should be (1) + } + + val attempt2 = newLogFile("app1", "attempt2", inProgress = true) + writeFile(attempt2, true, None, + SparkListenerApplicationStart("app1", Some("app1"), 3L, "test", "attempt2") + ) + + updateAndCheck(provider) { list => + list.size should be (1) + list.head.attempts.size should be (2) + list.head.attempts.head.attemptId should be ("attempt1") + } + + val completedAttempt2 = newLogFile("app1", "attempt2", inProgress = false) + attempt2.delete() + writeFile(attempt2, true, None, + SparkListenerApplicationStart("app1", Some("app1"), 3L, "test", "attempt2"), + SparkListenerApplicationEnd(4L) + ) + + updateAndCheck(provider) { list => + list should not be (null) + list.size should be (1) + list.head.attempts.size should be (2) + list.head.attempts.head.attemptId should be ("attempt2") + } + + val app2Attempt1 = newLogFile("app2", "attempt1", inProgress = false) + writeFile(attempt2, true, None, + SparkListenerApplicationStart("app2", Some("app2"), 5L, "test", "attempt1"), + SparkListenerApplicationEnd(6L) + ) + + updateAndCheck(provider) { list => + list.size should be (2) + list.head.attempts.size should be (1) + list.last.attempts.size should be (2) + list.head.attempts.head.attemptId should be ("attempt1") + + list.foreach { case app => + app.attempts.foreach { attempt => + val appUi = provider.getAppUI(app.id, attempt.attemptId) + appUi should not be null + } + } + + } + } + + /** + * Asks the provider to check for logs and calls a function to perform checks on the updated + * app list. Example: + * + * updateAndCheck(provider) { list => + * // asserts + * } + */ + private def updateAndCheck(provider: FsHistoryProvider) + (checkFn: Seq[ApplicationHistoryInfo] => Unit): Unit = { + provider.checkForLogs() + checkFn(provider.getListing().toSeq) + } + private def writeFile(file: File, isNewFormat: Boolean, codec: Option[CompressionCodec], events: SparkListenerEvent*) = { val fstream = new FileOutputStream(file) From cbe8bba9b0ad62f51fa4a742ac2b2665f730ffd1 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 8 Apr 2015 15:01:28 -0700 Subject: [PATCH 08/22] Attempt ID in listener event should be an option. For backwards compatibility. --- .../scala/org/apache/spark/SparkContext.scala | 8 +++--- .../scheduler/ApplicationEventListener.scala | 2 +- .../spark/scheduler/SparkListener.scala | 4 +-- .../org/apache/spark/util/JsonProtocol.scala | 2 +- .../history/FsHistoryProviderSuite.scala | 28 +++++++++---------- .../scheduler/EventLoggingListenerSuite.scala | 2 +- .../spark/scheduler/ReplayListenerSuite.scala | 2 +- .../apache/spark/util/JsonProtocolSuite.scala | 5 ++-- 8 files changed, 27 insertions(+), 26 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 3ed1f8157e496..64847486efa9c 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1404,17 +1404,17 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli def stop() { // Use the stopping variable to ensure no contention for the stop scenario. // Still track the stopped variable for use elsewhere in the code. - + if (!stopped.compareAndSet(false, true)) { logInfo("SparkContext already stopped.") return } - + postApplicationEnd() ui.foreach(_.stop()) env.metricsSystem.report() metadataCleaner.cancel() - cleaner.foreach(_.stop()) + cleaner.foreach(_.stop()) executorAllocationManager.foreach(_.stop()) dagScheduler.stop() dagScheduler = null @@ -1757,7 +1757,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli // Note: this code assumes that the task scheduler has been initialized and has contacted // the cluster manager to get an application ID (in case the cluster manager provides one). listenerBus.post(SparkListenerApplicationStart(appName, Some(applicationId), - startTime, sparkUser, applicationAttemptId)) + startTime, sparkUser, Some(applicationAttemptId))) } /** Post the application end event */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala index a591c7e046d5d..9f218c64cac2d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala @@ -36,7 +36,7 @@ private[spark] class ApplicationEventListener extends SparkListener { override def onApplicationStart(applicationStart: SparkListenerApplicationStart) { appName = Some(applicationStart.appName) appId = applicationStart.appId - appAttemptId = Some(applicationStart.appAttemptId) + appAttemptId = applicationStart.appAttemptId startTime = Some(applicationStart.time) sparkUser = Some(applicationStart.sparkUser) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 9f526b08a7b42..169d4fd3a94f0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -110,8 +110,8 @@ case class SparkListenerExecutorMetricsUpdate( extends SparkListenerEvent @DeveloperApi -case class SparkListenerApplicationStart(appName: String, appId: Option[String], - time: Long, sparkUser: String, appAttemptId: String = "") extends SparkListenerEvent +case class SparkListenerApplicationStart(appName: String, appId: Option[String], + time: Long, sparkUser: String, appAttemptId: Option[String]) extends SparkListenerEvent @DeveloperApi case class SparkListenerApplicationEnd(time: Long) extends SparkListenerEvent diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 9dbf466278eb0..23c18edf70b25 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -563,7 +563,7 @@ private[spark] object JsonProtocol { val appId = Utils.jsonOption(json \ "App ID").map(_.extract[String]) val time = (json \ "Timestamp").extract[Long] val sparkUser = (json \ "User").extract[String] - val appAttemptId = (json \ "appAttemptId").extract[String] + val appAttemptId = Utils.jsonOption(json \ "appAttemptId").map(_.extract[String]) SparkListenerApplicationStart(appName, appId, time, sparkUser, appAttemptId) } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index 78ab93247d8c7..8c0dc6a0da688 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -62,20 +62,20 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers // Write a new-style application log. val newAppComplete = newLogFile("new1", "", inProgress = false) writeFile(newAppComplete, true, None, - SparkListenerApplicationStart("new-app-complete", None, 1L, "test"), + SparkListenerApplicationStart("new-app-complete", None, 1L, "test", None), SparkListenerApplicationEnd(5L) ) // Write a new-style application log. val newAppCompressedComplete = newLogFile("new1compressed", "", inProgress = false, Some("lzf")) writeFile(newAppCompressedComplete, true, None, - SparkListenerApplicationStart("new-app-compressed-complete", None, 1L, "test"), + SparkListenerApplicationStart("new-app-compressed-complete", None, 1L, "test", None), SparkListenerApplicationEnd(4L)) // Write an unfinished app, new-style. val newAppIncomplete = newLogFile("new2", "", inProgress = true) writeFile(newAppIncomplete, true, None, - SparkListenerApplicationStart("new-app-incomplete", None, 1L, "test") + SparkListenerApplicationStart("new-app-incomplete", None, 1L, "test", None) ) // Write an old-style application log. @@ -83,7 +83,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers oldAppComplete.mkdir() createEmptyFile(new File(oldAppComplete, provider.SPARK_VERSION_PREFIX + "1.0")) writeFile(new File(oldAppComplete, provider.LOG_PREFIX + "1"), false, None, - SparkListenerApplicationStart("old-app-complete", None, 2L, "test"), + SparkListenerApplicationStart("old-app-complete", None, 2L, "test", None), SparkListenerApplicationEnd(3L) ) createEmptyFile(new File(oldAppComplete, provider.APPLICATION_COMPLETE)) @@ -97,7 +97,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers oldAppIncomplete.mkdir() createEmptyFile(new File(oldAppIncomplete, provider.SPARK_VERSION_PREFIX + "1.0")) writeFile(new File(oldAppIncomplete, provider.LOG_PREFIX + "1"), false, None, - SparkListenerApplicationStart("old-app-incomplete", None, 2L, "test") + SparkListenerApplicationStart("old-app-incomplete", None, 2L, "test", None) ) // Force a reload of data from the log directory, and check that both logs are loaded. @@ -144,7 +144,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers logDir.mkdir() createEmptyFile(new File(logDir, provider.SPARK_VERSION_PREFIX + "1.0")) writeFile(new File(logDir, provider.LOG_PREFIX + "1"), false, Option(codec), - SparkListenerApplicationStart("app2", None, 2L, "test"), + SparkListenerApplicationStart("app2", None, 2L, "test", None), SparkListenerApplicationEnd(3L) ) createEmptyFile(new File(logDir, provider.COMPRESSION_CODEC_PREFIX + codecName)) @@ -167,12 +167,12 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers test("SPARK-3697: ignore directories that cannot be read.") { val logFile1 = newLogFile("new1", "", inProgress = false) writeFile(logFile1, true, None, - SparkListenerApplicationStart("app1-1", None, 1L, "test"), + SparkListenerApplicationStart("app1-1", None, 1L, "test", None), SparkListenerApplicationEnd(2L) ) val logFile2 = newLogFile("new2", "", inProgress = false) writeFile(logFile2, true, None, - SparkListenerApplicationStart("app1-2", None, 1L, "test"), + SparkListenerApplicationStart("app1-2", None, 1L, "test", None), SparkListenerApplicationEnd(2L) ) logFile2.setReadable(false, false) @@ -188,7 +188,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers val logFile1 = newLogFile("app1", "", inProgress = true) writeFile(logFile1, true, None, - SparkListenerApplicationStart("app1", Some("app1"), 1L, "test"), + SparkListenerApplicationStart("app1", Some("app1"), 1L, "test", None), SparkListenerApplicationEnd(2L) ) updateAndCheck(provider) { list => @@ -210,7 +210,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers val logFile1 = newLogFile("app1", "", inProgress = true) writeFile(logFile1, true, None, - SparkListenerApplicationStart("app1", Some("app1"), 1L, "test"), + SparkListenerApplicationStart("app1", Some("app1"), 1L, "test", None), SparkListenerApplicationEnd(2L)) val oldLog = new File(testDir, "old1") @@ -226,7 +226,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers val attempt1 = newLogFile("app1", "attempt1", inProgress = false) writeFile(attempt1, true, None, - SparkListenerApplicationStart("app1", Some("app1"), 1L, "test", "attempt1"), + SparkListenerApplicationStart("app1", Some("app1"), 1L, "test", Some("attempt1")), SparkListenerApplicationEnd(2L) ) @@ -237,7 +237,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers val attempt2 = newLogFile("app1", "attempt2", inProgress = true) writeFile(attempt2, true, None, - SparkListenerApplicationStart("app1", Some("app1"), 3L, "test", "attempt2") + SparkListenerApplicationStart("app1", Some("app1"), 3L, "test", Some("attempt2")) ) updateAndCheck(provider) { list => @@ -249,7 +249,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers val completedAttempt2 = newLogFile("app1", "attempt2", inProgress = false) attempt2.delete() writeFile(attempt2, true, None, - SparkListenerApplicationStart("app1", Some("app1"), 3L, "test", "attempt2"), + SparkListenerApplicationStart("app1", Some("app1"), 3L, "test", Some("attempt2")), SparkListenerApplicationEnd(4L) ) @@ -262,7 +262,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers val app2Attempt1 = newLogFile("app2", "attempt1", inProgress = false) writeFile(attempt2, true, None, - SparkListenerApplicationStart("app2", Some("app2"), 5L, "test", "attempt1"), + SparkListenerApplicationStart("app2", Some("app2"), 5L, "test", Some("attempt1")), SparkListenerApplicationEnd(6L) ) diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index 2c3efd8b71e2a..e9b59a0041f52 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -143,7 +143,7 @@ class EventLoggingListenerSuite extends FunSuite with LocalSparkContext with Bef val eventLogger = new EventLoggingListener(logName, testDirPath.toUri(), conf) val listenerBus = new LiveListenerBus val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", None, - 125L, "Mickey") + 125L, "Mickey", None) val applicationEnd = SparkListenerApplicationEnd(1000L) // A comprehensive test on JSON de/serialization of all events is in JsonProtocolSuite diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala index 6de6d2fec622a..cb94e8d1aeeb4 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -50,7 +50,7 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { val fstream = fileSystem.create(logFilePath) val writer = new PrintWriter(fstream) val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", None, - 125L, "Mickey") + 125L, "Mickey", None) val applicationEnd = SparkListenerApplicationEnd(1000L) writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationStart)))) writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationEnd)))) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index a2be724254d7c..09b604a95a309 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -74,7 +74,8 @@ class JsonProtocolSuite extends FunSuite { val blockManagerRemoved = SparkListenerBlockManagerRemoved(2L, BlockManagerId("Scarce", "to be counted...", 100)) val unpersistRdd = SparkListenerUnpersistRDD(12345) - val applicationStart = SparkListenerApplicationStart("The winner of all", None, 42L, "Garfield") + val applicationStart = SparkListenerApplicationStart("The winner of all", None, 42L, "Garfield", + None) val applicationEnd = SparkListenerApplicationEnd(42L) val logUrlMap = Map("stderr" -> "mystderr", "stdout" -> "mystdout").toMap val executorAdded = SparkListenerExecutorAdded(executorAddedTime, "exec1", @@ -274,7 +275,7 @@ class JsonProtocolSuite extends FunSuite { test("SparkListenerApplicationStart backwards compatibility") { // SparkListenerApplicationStart in Spark 1.0.0 do not have an "appId" property. - val applicationStart = SparkListenerApplicationStart("test", None, 1L, "user") + val applicationStart = SparkListenerApplicationStart("test", None, 1L, "user", None) val oldEvent = JsonProtocol.applicationStartToJson(applicationStart) .removeField({ _._1 == "App ID" }) assert(applicationStart === JsonProtocol.applicationStartFromJson(oldEvent)) From ce5ee5de4206d74491ecd13411ca2947a81ab4e1 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 8 Apr 2015 16:57:21 -0700 Subject: [PATCH 09/22] Misc UI, test, style fixes. --- .../spark/deploy/history/HistoryPage.scala | 24 ++++++++++++------- .../spark/deploy/history/HistoryServer.scala | 11 ++++++--- .../scheduler/EventLoggingListener.scala | 23 ++++++++++++------ .../spark/scheduler/TaskScheduler.scala | 4 ++-- .../org/apache/spark/util/JsonProtocol.scala | 2 +- .../scheduler/EventLoggingListenerSuite.scala | 4 ++-- .../spark/deploy/yarn/ApplicationMaster.scala | 7 +++--- .../cluster/YarnClusterSchedulerBackend.scala | 2 +- 8 files changed, 48 insertions(+), 29 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index e51256c40122b..b6da1e09444fc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -160,6 +160,7 @@ private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") } private def attemptRow( + renderAttemptIdColumn: Boolean, info: ApplicationHistoryInfo, attempt: ApplicationAttemptInfo, isFirst: Boolean): Seq[Node] = { @@ -178,22 +179,27 @@ private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { if (isFirst) { if (info.attempts.size > 1) { - {info.id} + {info.id} ++ + {attempt.name} } else { - {info.id} + {info.id} ++ + {attempt.name} } } else { - new xml.Comment("") + Nil } } { - if (info.attempts.size > 1 && !attempt.attemptId.isEmpty) { - {attempt.attemptId} + if (renderAttemptIdColumn) { + if (info.attempts.size > 1 && !attempt.attemptId.isEmpty) { + {attempt.attemptId} + } else { +   + } } else { Nil } } - {attempt.name} {startTime} {endTime} @@ -204,12 +210,12 @@ private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") } private def appRow(info: ApplicationHistoryInfo): Seq[Node] = { - attemptRow(info, info.attempts.head, true) + attemptRow(false, info, info.attempts.head, true) } private def appWithAttemptRow(info: ApplicationHistoryInfo): Seq[Node] = { - attemptRow(info, info.attempts.head, true) ++ - info.attempts.drop(1).flatMap(attemptRow(info, _, false)) + attemptRow(true, info, info.attempts.head, true) ++ + info.attempts.drop(1).flatMap(attemptRow(true, info, _, false)) } private def makePageLink(linkPage: Int, showIncomplete: Boolean): String = { diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index b7b4355351619..0761d2846b854 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -80,18 +80,23 @@ class HistoryServer( return } - val appId = parts(1) + val appKey = + if (parts.length == 3) { + s"${parts(1)}/${parts(2)}" + } else { + parts(1) + } // Note we don't use the UI retrieved from the cache; the cache loader above will register // the app's UI, and all we need to do is redirect the user to the same URI that was // requested, and the proper data should be served at that point. try { - appCache.get(appId) + appCache.get(appKey) res.sendRedirect(res.encodeRedirectURL(req.getRequestURI())) } catch { case e: Exception => e.getCause() match { case nsee: NoSuchElementException => - val msg =
Application {appId} not found.
+ val msg =
Application {appKey} not found.
res.setStatus(HttpServletResponse.SC_NOT_FOUND) UIUtils.basicSparkPage(msg, "Not Found").foreach( n => res.getWriter().write(n.toString)) diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index c75c7a53afefa..238694324c19f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -55,8 +55,8 @@ private[spark] class EventLoggingListener( import EventLoggingListener._ - def this(appId: String, logBaseDir: URI, sparkConf: SparkConf) = - this(appId, "", logBaseDir, sparkConf, SparkHadoopUtil.get.newConfiguration(sparkConf)) + def this(appId: String, appAttemptId : String, logBaseDir: URI, sparkConf: SparkConf) = + this(appId, appAttemptId, logBaseDir, sparkConf, SparkHadoopUtil.get.newConfiguration(sparkConf)) private val shouldCompress = sparkConf.getBoolean("spark.eventLog.compress", false) private val shouldOverwrite = sparkConf.getBoolean("spark.eventLog.overwrite", false) @@ -253,9 +253,12 @@ private[spark] object EventLoggingListener extends Logging { * we won't know which codec to use to decompress the metadata needed to open the file in * the first place. * + * The log file name will identify the compression codec used for the contents, if any. + * For example, app_123 for an uncompressed log, app_123.lzf for an LZF-compressed log. + * * @param logBaseDir Directory where the log file will be written. * @param appId A unique app ID. - * @param appAttemptId A unique attempt id of appId. + * @param appAttemptId A unique attempt id of appId. May be the empty string. * @param compressionCodecName Name to identify the codec used to compress the contents * of the log, or None if compression is not enabled. * @return A path which consists of file-system-safe characters. @@ -265,14 +268,20 @@ private[spark] object EventLoggingListener extends Logging { appId: String, appAttemptId: String, compressionCodecName: Option[String] = None): String = { - val name = appId.replaceAll("[ :/]", "-").replaceAll("[${}'\"]", "_").toLowerCase - if (appAttemptId.equals("")) { - logBaseDir.toString.stripSuffix("/") + "/" + name.stripSuffix("/") + val sanitizedAppId = appId.replaceAll("[ :/]", "-").replaceAll("[.${}'\"]", "_").toLowerCase + val base = logBaseDir.toString.stripSuffix("/") + "/" + sanitize(appId) + val codec = compressionCodecName.map("." + _).getOrElse("") + if (appAttemptId.isEmpty) { + base + codec } else { - logBaseDir.toString.stripSuffix("/") + "/" + name.stripSuffix("/") + "_" + appAttemptId + base + "_" + sanitize(appAttemptId) + codec } } + private def sanitize(str: String): String = { + str.replaceAll("[ :/]", "-").replaceAll("[.${}'\"]", "_").toLowerCase + } + /** * Opens an event log file and returns an input stream that contains the event data. * diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index d612409c81b6f..9a64629eeb885 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -73,14 +73,14 @@ private[spark] trait TaskScheduler { * @return An application ID */ def applicationId(): String = appId - + /** * Process a lost executor */ def executorLost(executorId: String, reason: ExecutorLossReason): Unit /** - * Get an application's attempt Id associated with the job. + * Get an application's attempt ID associated with the job. * * @return An application's Attempt ID */ diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 23c18edf70b25..0a877a78714b2 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -195,7 +195,7 @@ private[spark] object JsonProtocol { ("App ID" -> applicationStart.appId.map(JString(_)).getOrElse(JNothing)) ~ ("Timestamp" -> applicationStart.time) ~ ("User" -> applicationStart.sparkUser) ~ - ("appAttemptId" -> applicationStart.appAttemptId) + ("appAttemptId" -> applicationStart.appAttemptId.map(JString(_)).getOrElse(JNothing)) } def applicationEndToJson(applicationEnd: SparkListenerApplicationEnd): JValue = { diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index e9b59a0041f52..1d6c5c868e8f4 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -61,7 +61,7 @@ class EventLoggingListenerSuite extends FunSuite with LocalSparkContext with Bef test("Verify log file exist") { // Verify logging directory exists val conf = getLoggingConf(testDirPath) - val eventLogger = new EventLoggingListener("test", testDirPath.toUri(), conf) + val eventLogger = new EventLoggingListener("test", "", testDirPath.toUri(), conf) eventLogger.start() val logPath = new Path(eventLogger.logPath + EventLoggingListener.IN_PROGRESS) @@ -140,7 +140,7 @@ class EventLoggingListenerSuite extends FunSuite with LocalSparkContext with Bef val conf = getLoggingConf(testDirPath, compressionCodec) extraConf.foreach { case (k, v) => conf.set(k, v) } val logName = compressionCodec.map("test-" + _).getOrElse("test") - val eventLogger = new EventLoggingListener(logName, testDirPath.toUri(), conf) + val eventLogger = new EventLoggingListener(logName, "", testDirPath.toUri(), conf) val listenerBus = new LiveListenerBus val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", None, 125L, "Mickey", None) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 3cc2a9f5570b1..e14a70d353a3f 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -91,10 +91,9 @@ private[spark] class ApplicationMaster( // Propagate the application ID so that YarnClusterSchedulerBackend can pick it up. System.setProperty("spark.yarn.app.id", appAttemptId.getApplicationId().toString()) - // Propagate the attempt if, so that in case of event logging, - // different attempt's logs gets created in different directory - System.setProperty("spark.yarn.app.attemptid", appAttemptId.getAttemptId().toString()) - + // Propagate the attempt if, so that in case of event logging, + // different attempt's logs gets created in different directory + System.setProperty("spark.yarn.app.attemptid", appAttemptId.getAttemptId().toString()) } logInfo("ApplicationAttemptId: " + appAttemptId) diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala index f20f4dcb00d64..f022e8296fa66 100644 --- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala @@ -46,7 +46,7 @@ private[spark] class YarnClusterSchedulerBackend( logError("Application ID is not set.") super.applicationId } - + override def applicationAttemptId(): String = // In YARN Cluster mode, spark.yarn.app.attemptid is expect to be set // before user application is launched. From c3e0a828a975053144479d023d3b639492119ac6 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 8 Apr 2015 17:15:47 -0700 Subject: [PATCH 10/22] Move app name to app info, more UI fixes. --- .../history/ApplicationHistoryProvider.scala | 2 +- .../deploy/history/FsHistoryProvider.scala | 20 ++++++++++--------- .../spark/deploy/history/HistoryPage.scala | 17 ++++++++-------- .../history/FsHistoryProviderSuite.scala | 4 ++-- .../deploy/history/HistoryServerSuite.scala | 4 ++-- .../spark/scheduler/ReplayListenerSuite.scala | 2 +- 6 files changed, 25 insertions(+), 24 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala index 166afc97a6c12..9291d66e14cd4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala @@ -21,7 +21,6 @@ import org.apache.spark.ui.SparkUI private[history] case class ApplicationAttemptInfo( attemptId: String, - name: String, startTime: Long, endTime: Long, lastUpdated: Long, @@ -30,6 +29,7 @@ private[history] case class ApplicationAttemptInfo( private[history] case class ApplicationHistoryInfo( id: String, + name: String, attempts: List[ApplicationAttemptInfo]) private[history] abstract class ApplicationHistoryProvider { diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 2798ec0237619..21d2eb95c5eb7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -145,8 +145,8 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis override def getAppUI(appId: String, attemptId: String): Option[SparkUI] = { try { - applications.get(appId).flatMap { info => - val attempts = info.attempts.filter(_.attemptId == attemptId) + applications.get(appId).flatMap { appInfo => + val attempts = appInfo.attempts.filter(_.attemptId == attemptId) attempts.headOption.map { attempt => val replayBus = new ReplayListenerBus() val ui = { @@ -161,7 +161,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis replayBus.addListener(appListener) val appInfo = replay(fs.getFileStatus(new Path(logDir, attempt.logPath)), replayBus) - ui.setAppName(s"${attempt.name} ($appId)") + ui.setAppName(s"${appInfo.name} ($appId)") val uiAclsEnabled = conf.getBoolean("spark.history.ui.acls.enable", false) ui.getSecurityManager.setAcls(uiAclsEnabled) @@ -256,9 +256,10 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis .map { app => val attempts = app.attempts.filter(_.attemptId != attempt.attemptId).toList ++ List(attempt) - new FsApplicationHistoryInfo(attempt.appId, attempts.sortWith(compareAttemptInfo)) + new FsApplicationHistoryInfo(attempt.appId, attempt.name, + attempts.sortWith(compareAttemptInfo)) } - .getOrElse(new FsApplicationHistoryInfo(attempt.appId, List(attempt))) + .getOrElse(new FsApplicationHistoryInfo(attempt.appId, attempt.name, List(attempt))) newAppMap(attempt.appId) = appInfo } @@ -363,9 +364,9 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis bus.replay(logInput, logPath.toString) new FsApplicationAttemptInfo( logPath.getName(), + appListener.appName.getOrElse(NOT_STARTED), appListener.appId.getOrElse(logPath.getName()), appListener.appAttemptId.getOrElse(""), - appListener.appName.getOrElse(NOT_STARTED), appListener.startTime.getOrElse(-1L), appListener.endTime.getOrElse(-1L), getModificationTime(eventLog).get, @@ -459,18 +460,19 @@ private object FsHistoryProvider { private class FsApplicationAttemptInfo( val logPath: String, + val name: String, val appId: String, attemptId: String, - name: String, startTime: Long, endTime: Long, lastUpdated: Long, sparkUser: String, completed: Boolean = true) extends ApplicationAttemptInfo( - attemptId, name, startTime, endTime, lastUpdated, sparkUser, completed) + attemptId, startTime, endTime, lastUpdated, sparkUser, completed) private class FsApplicationHistoryInfo( id: String, + override val name: String, override val attempts: List[FsApplicationAttemptInfo]) - extends ApplicationHistoryInfo(id, attempts) + extends ApplicationHistoryInfo(id, name, attempts) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index b6da1e09444fc..40c3d91d1c95c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -164,26 +164,25 @@ private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") info: ApplicationHistoryInfo, attempt: ApplicationAttemptInfo, isFirst: Boolean): Seq[Node] = { - val attemptInfo = info.attempts.head - val uiAddress = HistoryServer.UI_PATH_PREFIX + s"/${info.id}" - val startTime = UIUtils.formatDate(attemptInfo.startTime) - val endTime = if (attemptInfo.endTime > 0) UIUtils.formatDate(attemptInfo.endTime) else "-" + val uiAddress = getAttemptURI(info.id, attempt) + val startTime = UIUtils.formatDate(attempt.startTime) + val endTime = if (attempt.endTime > 0) UIUtils.formatDate(attempt.endTime) else "-" val duration = - if (attemptInfo.endTime > 0) { - UIUtils.formatDuration(attemptInfo.endTime - attemptInfo.startTime) + if (attempt.endTime > 0) { + UIUtils.formatDuration(attempt.endTime - attempt.startTime) } else { "-" } - val lastUpdated = UIUtils.formatDate(attemptInfo.lastUpdated) + val lastUpdated = UIUtils.formatDate(attempt.lastUpdated) { if (isFirst) { if (info.attempts.size > 1) { {info.id} ++ - {attempt.name} + {info.name} } else { {info.id} ++ - {attempt.name} + {info.name} } } else { Nil diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index 8c0dc6a0da688..13eb2dce087e3 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -108,8 +108,8 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers def makeAppInfo(id: String, name: String, start: Long, end: Long, lastMod: Long, user: String, completed: Boolean): ApplicationHistoryInfo = { - ApplicationHistoryInfo(id, - List(ApplicationAttemptInfo("", name, start, end, lastMod, user, completed))) + ApplicationHistoryInfo(id, name, + List(ApplicationAttemptInfo("", start, end, lastMod, user, completed))) } list(0) should be (makeAppInfo(newAppComplete.getName(), "new-app-complete", 1L, 5L, diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index b750a9e49d1be..c5ebe34c60b3b 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -36,8 +36,8 @@ class HistoryServerSuite extends FunSuite with Matchers with MockitoSugar { val request = mock[HttpServletRequest] val ui = mock[SparkUI] val link = "/history/app1" - val info = new ApplicationHistoryInfo("app1", - List(ApplicationAttemptInfo("attempt1", "app1", 0, 2, 1, "xxx", true))) + val info = new ApplicationHistoryInfo("app1", "app1", + List(ApplicationAttemptInfo("attempt1", 0, 2, 1, "xxx", true))) when(historyServer.getApplicationList()).thenReturn(Seq(info)) when(ui.basePath).thenReturn(link) when(historyServer.getProviderConfig()).thenReturn(Map[String, String]()) diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala index cb94e8d1aeeb4..d387147ccf987 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -146,7 +146,7 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { * log the events. */ private class EventMonster(conf: SparkConf) - extends EventLoggingListener("test", new URI("testdir"), conf) { + extends EventLoggingListener("test", "", new URI("testdir"), conf) { override def start() { } From 657ec18b5affa0a3f77f51a3e3e05ddbbc21c85f Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 8 Apr 2015 17:27:26 -0700 Subject: [PATCH 11/22] Fix yarn history URL, app links. --- .../spark/deploy/history/FsHistoryProvider.scala | 2 +- .../org/apache/spark/deploy/history/HistoryPage.scala | 10 +++------- .../apache/spark/deploy/history/HistoryServer.scala | 5 +++++ .../apache/spark/deploy/yarn/ApplicationMaster.scala | 3 ++- 4 files changed, 11 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 21d2eb95c5eb7..783e144a8c7b4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -153,7 +153,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis val conf = this.conf.clone() val appSecManager = new SecurityManager(conf) SparkUI.createHistoryUI(conf, replayBus, appSecManager, appId, - s"${HistoryServer.UI_PATH_PREFIX}/$appId") + HistoryServer.getAttemptURI(appId, attempt.attemptId)) // Do not call ui.bind() to avoid creating a new server for each application } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index 40c3d91d1c95c..3182a216dd4c3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -154,17 +154,12 @@ private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") {nextPage} ) } - private def getAttemptURI(appId: String, attemptInfo: ApplicationAttemptInfo): String = { - val attemptSuffix = if (!attemptInfo.attemptId.isEmpty) s"/${attemptInfo.attemptId}" else "" - s"${HistoryServer.UI_PATH_PREFIX}/${appId}${attemptSuffix}" - } - private def attemptRow( renderAttemptIdColumn: Boolean, info: ApplicationHistoryInfo, attempt: ApplicationAttemptInfo, isFirst: Boolean): Seq[Node] = { - val uiAddress = getAttemptURI(info.id, attempt) + val uiAddress = HistoryServer.getAttemptURI(info.id, attempt.attemptId) val startTime = UIUtils.formatDate(attempt.startTime) val endTime = if (attempt.endTime > 0) UIUtils.formatDate(attempt.endTime) else "-" val duration = @@ -191,7 +186,8 @@ private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { if (renderAttemptIdColumn) { if (info.attempts.size > 1 && !attempt.attemptId.isEmpty) { - {attempt.attemptId} + + {attempt.attemptId} } else {   } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 0761d2846b854..830edc566a061 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -224,4 +224,9 @@ object HistoryServer extends Logging { } } + private[history] def getAttemptURI(appId: String, attemptId: String): String = { + val attemptSuffix = if (!attemptId.isEmpty) s"/${attemptId}" else "" + s"${HistoryServer.UI_PATH_PREFIX}/${appId}${attemptSuffix}" + } + } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index e14a70d353a3f..c8f56bba6d3bc 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -225,9 +225,10 @@ private[spark] class ApplicationMaster( val sc = sparkContextRef.get() val appId = client.getAttemptId().getApplicationId().toString() + val attemptId = client.getAttemptId().getAttemptId().toString() val historyAddress = sparkConf.getOption("spark.yarn.historyServer.address") - .map { address => s"${address}${HistoryServer.UI_PATH_PREFIX}/${appId}" } + .map { address => s"${address}${HistoryServer.UI_PATH_PREFIX}/${appId}/${attemptId}" } .getOrElse("") allocator = client.register(yarnConf, From 3a145030c7a2f729ce9665fdd454a83b21750c16 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 8 Apr 2015 17:50:46 -0700 Subject: [PATCH 12/22] Argh scalastyle. --- .../org/apache/spark/scheduler/EventLoggingListener.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 238694324c19f..a1c7b755eb3c1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -56,7 +56,8 @@ private[spark] class EventLoggingListener( import EventLoggingListener._ def this(appId: String, appAttemptId : String, logBaseDir: URI, sparkConf: SparkConf) = - this(appId, appAttemptId, logBaseDir, sparkConf, SparkHadoopUtil.get.newConfiguration(sparkConf)) + this(appId, appAttemptId, logBaseDir, sparkConf, + SparkHadoopUtil.get.newConfiguration(sparkConf)) private val shouldCompress = sparkConf.getBoolean("spark.eventLog.compress", false) private val shouldOverwrite = sparkConf.getBoolean("spark.eventLog.overwrite", false) From 9092af517c07504a275f7d6ae2187fe97f8c83cd Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 9 Apr 2015 10:10:17 -0700 Subject: [PATCH 13/22] Fix HistoryServer test. --- .../org/apache/spark/deploy/history/HistoryServerSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index c5ebe34c60b3b..2125cdff87573 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -37,7 +37,7 @@ class HistoryServerSuite extends FunSuite with Matchers with MockitoSugar { val ui = mock[SparkUI] val link = "/history/app1" val info = new ApplicationHistoryInfo("app1", "app1", - List(ApplicationAttemptInfo("attempt1", 0, 2, 1, "xxx", true))) + List(ApplicationAttemptInfo("", 0, 2, 1, "xxx", true))) when(historyServer.getApplicationList()).thenReturn(Seq(info)) when(ui.basePath).thenReturn(link) when(historyServer.getProviderConfig()).thenReturn(Map[String, String]()) From 07446c688f94ccdb98645b2e75960ab745caa339 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 10 Apr 2015 13:06:07 -0700 Subject: [PATCH 14/22] Disable striping for app id / name when multiple attempts exist. --- .../org/apache/spark/deploy/history/HistoryPage.scala | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index 3182a216dd4c3..0a41108a36b92 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -172,11 +172,13 @@ private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { if (isFirst) { - if (info.attempts.size > 1) { - {info.id} ++ - {info.name} + if (info.attempts.size > 1 || renderAttemptIdColumn) { + + {info.id} + + {info.name} } else { - {info.id} ++ + {info.id} {info.name} } } else { From ba34b69c3c60da990e69ebc82d60cdb215e70f79 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 23 Apr 2015 11:18:25 -0700 Subject: [PATCH 15/22] Use Option[String] for attempt id. --- .../scala/org/apache/spark/SparkContext.scala | 32 +++++++++---------- .../apache/spark/deploy/master/Master.scala | 2 +- .../scheduler/EventLoggingListener.scala | 12 +++---- .../spark/scheduler/SchedulerBackend.scala | 2 +- .../spark/scheduler/TaskScheduler.scala | 2 +- .../spark/scheduler/TaskSchedulerImpl.scala | 4 +-- .../history/FsHistoryProviderSuite.scala | 26 +++++++-------- .../spark/scheduler/DAGSchedulerSuite.scala | 2 ++ .../scheduler/EventLoggingListenerSuite.scala | 16 +++++----- .../spark/scheduler/ReplayListenerSuite.scala | 2 +- .../cluster/YarnClusterSchedulerBackend.scala | 4 +-- 11 files changed, 53 insertions(+), 51 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index b9392aecbaadb..9a09f5a809838 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -217,7 +217,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli private var _heartbeatReceiver: RpcEndpointRef = _ @volatile private var _dagScheduler: DAGScheduler = _ private var _applicationId: String = _ - private var _applicationAttemptId: String = _ + private var _applicationAttemptId: Option[String] = None private var _eventLogger: Option[EventLoggingListener] = None private var _executorAllocationManager: Option[ExecutorAllocationManager] = None private var _cleaner: Option[ContextCleaner] = None @@ -313,7 +313,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli } def applicationId: String = _applicationId - def applicationAttemptId: String = _applicationAttemptId + def applicationAttemptId: Option[String] = _applicationAttemptId def metricsSystem: MetricsSystem = if (_env != null) _env.metricsSystem else null @@ -1847,7 +1847,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli // Note: this code assumes that the task scheduler has been initialized and has contacted // the cluster manager to get an application ID (in case the cluster manager provides one). listenerBus.post(SparkListenerApplicationStart(appName, Some(applicationId), - startTime, sparkUser, Some(applicationAttemptId))) + startTime, sparkUser, applicationAttemptId)) } /** Post the application end event */ @@ -1895,7 +1895,7 @@ object SparkContext extends Logging { * * Access to this field is guarded by SPARK_CONTEXT_CONSTRUCTOR_LOCK. */ - private val activeContext: AtomicReference[SparkContext] = + private val activeContext: AtomicReference[SparkContext] = new AtomicReference[SparkContext](null) /** @@ -1948,11 +1948,11 @@ object SparkContext extends Logging { } /** - * This function may be used to get or instantiate a SparkContext and register it as a - * singleton object. Because we can only have one active SparkContext per JVM, - * this is useful when applications may wish to share a SparkContext. + * This function may be used to get or instantiate a SparkContext and register it as a + * singleton object. Because we can only have one active SparkContext per JVM, + * this is useful when applications may wish to share a SparkContext. * - * Note: This function cannot be used to create multiple SparkContext instances + * Note: This function cannot be used to create multiple SparkContext instances * even if multiple contexts are allowed. */ def getOrCreate(config: SparkConf): SparkContext = { @@ -1965,17 +1965,17 @@ object SparkContext extends Logging { activeContext.get() } } - + /** - * This function may be used to get or instantiate a SparkContext and register it as a - * singleton object. Because we can only have one active SparkContext per JVM, + * This function may be used to get or instantiate a SparkContext and register it as a + * singleton object. Because we can only have one active SparkContext per JVM, * this is useful when applications may wish to share a SparkContext. - * + * * This method allows not passing a SparkConf (useful if just retrieving). - * - * Note: This function cannot be used to create multiple SparkContext instances - * even if multiple contexts are allowed. - */ + * + * Note: This function cannot be used to create multiple SparkContext instances + * even if multiple contexts are allowed. + */ def getOrCreate(): SparkContext = { getOrCreate(new SparkConf()) } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index a0e429e203918..b7b9d48e5dc87 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -760,7 +760,7 @@ private[master] class Master( } val eventLogFilePrefix = EventLoggingListener.getLogPath( - eventLogDir, app.id, "", app.desc.eventLogCodec) + eventLogDir, app.id, None, app.desc.eventLogCodec) val fs = Utils.getHadoopFileSystem(eventLogDir, hadoopConf) val inProgressExists = fs.exists(new Path(eventLogFilePrefix + EventLoggingListener.IN_PROGRESS)) diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index a1c7b755eb3c1..eba50b54fc7d8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -47,7 +47,7 @@ import org.apache.spark.util.{JsonProtocol, Utils} */ private[spark] class EventLoggingListener( appId: String, - appAttemptId : String, + appAttemptId : Option[String], logBaseDir: URI, sparkConf: SparkConf, hadoopConf: Configuration) @@ -55,7 +55,7 @@ private[spark] class EventLoggingListener( import EventLoggingListener._ - def this(appId: String, appAttemptId : String, logBaseDir: URI, sparkConf: SparkConf) = + def this(appId: String, appAttemptId : Option[String], logBaseDir: URI, sparkConf: SparkConf) = this(appId, appAttemptId, logBaseDir, sparkConf, SparkHadoopUtil.get.newConfiguration(sparkConf)) @@ -267,15 +267,15 @@ private[spark] object EventLoggingListener extends Logging { def getLogPath( logBaseDir: URI, appId: String, - appAttemptId: String, + appAttemptId: Option[String], compressionCodecName: Option[String] = None): String = { val sanitizedAppId = appId.replaceAll("[ :/]", "-").replaceAll("[.${}'\"]", "_").toLowerCase val base = logBaseDir.toString.stripSuffix("/") + "/" + sanitize(appId) val codec = compressionCodecName.map("." + _).getOrElse("") - if (appAttemptId.isEmpty) { - base + codec + if (appAttemptId.isDefined) { + base + "_" + sanitize(appAttemptId.get) + codec } else { - base + "_" + sanitize(appAttemptId) + codec + base + codec } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala index 6a6ab0c82f310..70500ccdb6212 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala @@ -46,6 +46,6 @@ private[spark] trait SchedulerBackend { * * @return An application attempt id */ - def applicationAttemptId(): String = "" + def applicationAttemptId(): Option[String] = None } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 9a64629eeb885..f25f3ed0d9037 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -84,6 +84,6 @@ private[spark] trait TaskScheduler { * * @return An application's Attempt ID */ - def applicationAttemptId(): String = "" + def applicationAttemptId(): Option[String] } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index b6a84d3542a5f..b4b8a630694bb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -512,8 +512,8 @@ private[spark] class TaskSchedulerImpl( } override def applicationId(): String = backend.applicationId() - - override def applicationAttemptId(): String = backend.applicationAttemptId() + + override def applicationAttemptId(): Option[String] = backend.applicationAttemptId() } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index 13eb2dce087e3..d3e49938a17cd 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -47,7 +47,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers /** Create a fake log file using the new log format used in Spark 1.3+ */ private def newLogFile( appId: String, - appAttemptId: String, + appAttemptId: Option[String], inProgress: Boolean, codec: Option[String] = None): File = { val ip = if (inProgress) EventLoggingListener.IN_PROGRESS else "" @@ -60,20 +60,20 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers val provider = new FsHistoryProvider(createTestConf()) // Write a new-style application log. - val newAppComplete = newLogFile("new1", "", inProgress = false) + val newAppComplete = newLogFile("new1", None, inProgress = false) writeFile(newAppComplete, true, None, SparkListenerApplicationStart("new-app-complete", None, 1L, "test", None), SparkListenerApplicationEnd(5L) ) // Write a new-style application log. - val newAppCompressedComplete = newLogFile("new1compressed", "", inProgress = false, Some("lzf")) + val newAppCompressedComplete = newLogFile("new1compressed", None, inProgress = false, Some("lzf")) writeFile(newAppCompressedComplete, true, None, SparkListenerApplicationStart("new-app-compressed-complete", None, 1L, "test", None), SparkListenerApplicationEnd(4L)) // Write an unfinished app, new-style. - val newAppIncomplete = newLogFile("new2", "", inProgress = true) + val newAppIncomplete = newLogFile("new2", None, inProgress = true) writeFile(newAppIncomplete, true, None, SparkListenerApplicationStart("new-app-incomplete", None, 1L, "test", None) ) @@ -165,12 +165,12 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers } test("SPARK-3697: ignore directories that cannot be read.") { - val logFile1 = newLogFile("new1", "", inProgress = false) + val logFile1 = newLogFile("new1", None, inProgress = false) writeFile(logFile1, true, None, SparkListenerApplicationStart("app1-1", None, 1L, "test", None), SparkListenerApplicationEnd(2L) ) - val logFile2 = newLogFile("new2", "", inProgress = false) + val logFile2 = newLogFile("new2", None, inProgress = false) writeFile(logFile2, true, None, SparkListenerApplicationStart("app1-2", None, 1L, "test", None), SparkListenerApplicationEnd(2L) @@ -186,7 +186,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers test("history file is renamed from inprogress to completed") { val provider = new FsHistoryProvider(createTestConf()) - val logFile1 = newLogFile("app1", "", inProgress = true) + val logFile1 = newLogFile("app1", None, inProgress = true) writeFile(logFile1, true, None, SparkListenerApplicationStart("app1", Some("app1"), 1L, "test", None), SparkListenerApplicationEnd(2L) @@ -197,7 +197,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers endWith(EventLoggingListener.IN_PROGRESS) } - logFile1.renameTo(newLogFile("app1", "", inProgress = false)) + logFile1.renameTo(newLogFile("app1", None, inProgress = false)) updateAndCheck(provider) { list => list.size should be (1) list.head.attempts.head.asInstanceOf[FsApplicationAttemptInfo].logPath should not @@ -208,7 +208,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers test("SPARK-5582: empty log directory") { val provider = new FsHistoryProvider(createTestConf()) - val logFile1 = newLogFile("app1", "", inProgress = true) + val logFile1 = newLogFile("app1", None, inProgress = true) writeFile(logFile1, true, None, SparkListenerApplicationStart("app1", Some("app1"), 1L, "test", None), SparkListenerApplicationEnd(2L)) @@ -224,7 +224,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers test("apps with multiple attempts") { val provider = new FsHistoryProvider(createTestConf()) - val attempt1 = newLogFile("app1", "attempt1", inProgress = false) + val attempt1 = newLogFile("app1", Some("attempt1"), inProgress = false) writeFile(attempt1, true, None, SparkListenerApplicationStart("app1", Some("app1"), 1L, "test", Some("attempt1")), SparkListenerApplicationEnd(2L) @@ -235,7 +235,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers list.head.attempts.size should be (1) } - val attempt2 = newLogFile("app1", "attempt2", inProgress = true) + val attempt2 = newLogFile("app1", Some("attempt2"), inProgress = true) writeFile(attempt2, true, None, SparkListenerApplicationStart("app1", Some("app1"), 3L, "test", Some("attempt2")) ) @@ -246,7 +246,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers list.head.attempts.head.attemptId should be ("attempt1") } - val completedAttempt2 = newLogFile("app1", "attempt2", inProgress = false) + val completedAttempt2 = newLogFile("app1", Some("attempt2"), inProgress = false) attempt2.delete() writeFile(attempt2, true, None, SparkListenerApplicationStart("app1", Some("app1"), 3L, "test", Some("attempt2")), @@ -260,7 +260,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers list.head.attempts.head.attemptId should be ("attempt2") } - val app2Attempt1 = newLogFile("app2", "attempt1", inProgress = false) + val app2Attempt1 = newLogFile("app2", Some("attempt1"), inProgress = false) writeFile(attempt2, true, None, SparkListenerApplicationStart("app2", Some("app2"), 5L, "test", Some("attempt1")), SparkListenerApplicationEnd(6L) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 3c52a8c4460c6..2482603f420ee 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -95,6 +95,7 @@ class DAGSchedulerSuite override def setDAGScheduler(dagScheduler: DAGScheduler) = {} override def defaultParallelism() = 2 override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {} + override def applicationAttemptId(): Option[String] = None } /** Length of time to wait while draining listener events. */ @@ -404,6 +405,7 @@ class DAGSchedulerSuite taskMetrics: Array[(Long, TaskMetrics)], blockManagerId: BlockManagerId): Boolean = true override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {} + override def applicationAttemptId(): Option[String] = None } val noKillScheduler = new DAGScheduler( sc, diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index cd8f0931485db..b52a8d11d147d 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -61,7 +61,7 @@ class EventLoggingListenerSuite extends FunSuite with LocalSparkContext with Bef test("Verify log file exist") { // Verify logging directory exists val conf = getLoggingConf(testDirPath) - val eventLogger = new EventLoggingListener("test", "", testDirPath.toUri(), conf) + val eventLogger = new EventLoggingListener("test", None, testDirPath.toUri(), conf) eventLogger.start() val logPath = new Path(eventLogger.logPath + EventLoggingListener.IN_PROGRESS) @@ -95,7 +95,7 @@ class EventLoggingListenerSuite extends FunSuite with LocalSparkContext with Bef } test("Log overwriting") { - val logUri = EventLoggingListener.getLogPath(testDir.toURI, "test", "") + val logUri = EventLoggingListener.getLogPath(testDir.toURI, "test", None) val logPath = new URI(logUri).getPath // Create file before writing the event log new FileOutputStream(new File(logPath)).close() @@ -108,18 +108,18 @@ class EventLoggingListenerSuite extends FunSuite with LocalSparkContext with Bef test("Event log name") { // without compression assert(s"file:/base-dir/app1" === EventLoggingListener.getLogPath( - Utils.resolveURI("/base-dir"), "app1", "")) + Utils.resolveURI("/base-dir"), "app1", None)) // with compression assert(s"file:/base-dir/app1.lzf" === - EventLoggingListener.getLogPath(Utils.resolveURI("/base-dir"), "app1", "", Some("lzf"))) + EventLoggingListener.getLogPath(Utils.resolveURI("/base-dir"), "app1", None, Some("lzf"))) // illegal characters in app ID assert(s"file:/base-dir/a-fine-mind_dollar_bills__1" === EventLoggingListener.getLogPath(Utils.resolveURI("/base-dir"), - "a fine:mind$dollar{bills}.1", "")) + "a fine:mind$dollar{bills}.1", None)) // illegal characters in app ID with compression assert(s"file:/base-dir/a-fine-mind_dollar_bills__1.lz4" === EventLoggingListener.getLogPath(Utils.resolveURI("/base-dir"), - "a fine:mind$dollar{bills}.1", "", Some("lz4"))) + "a fine:mind$dollar{bills}.1", None, Some("lz4"))) } /* ----------------- * @@ -140,7 +140,7 @@ class EventLoggingListenerSuite extends FunSuite with LocalSparkContext with Bef val conf = getLoggingConf(testDirPath, compressionCodec) extraConf.foreach { case (k, v) => conf.set(k, v) } val logName = compressionCodec.map("test-" + _).getOrElse("test") - val eventLogger = new EventLoggingListener(logName, "", testDirPath.toUri(), conf) + val eventLogger = new EventLoggingListener(logName, None, testDirPath.toUri(), conf) val listenerBus = new LiveListenerBus val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", None, 125L, "Mickey", None) @@ -186,7 +186,7 @@ class EventLoggingListenerSuite extends FunSuite with LocalSparkContext with Bef val eventLogPath = eventLogger.logPath val expectedLogDir = testDir.toURI() assert(eventLogPath === EventLoggingListener.getLogPath( - expectedLogDir, sc.applicationId, "", compressionCodec.map(CompressionCodec.getShortName))) + expectedLogDir, sc.applicationId, None, compressionCodec.map(CompressionCodec.getShortName))) // Begin listening for events that trigger asserts val eventExistenceListener = new EventExistenceListener(eventLogger) diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala index d387147ccf987..dabe4574b6456 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -146,7 +146,7 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { * log the events. */ private class EventMonster(conf: SparkConf) - extends EventLoggingListener("test", "", new URI("testdir"), conf) { + extends EventLoggingListener("test", None, new URI("testdir"), conf) { override def start() { } diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala index f022e8296fa66..28b212146553f 100644 --- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala @@ -47,11 +47,11 @@ private[spark] class YarnClusterSchedulerBackend( super.applicationId } - override def applicationAttemptId(): String = + override def applicationAttemptId(): Option[String] = // In YARN Cluster mode, spark.yarn.app.attemptid is expect to be set // before user application is launched. // So, if spark.yarn.app.id is not set, it is something wrong. - sc.getConf.getOption("spark.yarn.app.attemptid").getOrElse { + sc.getConf.getOption("spark.yarn.app.attemptid").orElse { logError("Application attempt ID is not set.") super.applicationAttemptId } From d5a9c37a00f3b0b5aa66c5f92c325fdf0ac05bf0 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 23 Apr 2015 11:20:51 -0700 Subject: [PATCH 16/22] Update JsonProtocol test, make property name consistent. --- core/src/main/scala/org/apache/spark/util/JsonProtocol.scala | 2 +- .../test/scala/org/apache/spark/util/JsonProtocolSuite.scala | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 0a877a78714b2..39387d0642cba 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -195,7 +195,7 @@ private[spark] object JsonProtocol { ("App ID" -> applicationStart.appId.map(JString(_)).getOrElse(JNothing)) ~ ("Timestamp" -> applicationStart.time) ~ ("User" -> applicationStart.sparkUser) ~ - ("appAttemptId" -> applicationStart.appAttemptId.map(JString(_)).getOrElse(JNothing)) + ("App Attempt ID" -> applicationStart.appAttemptId.map(JString(_)).getOrElse(JNothing)) } def applicationEndToJson(applicationEnd: SparkListenerApplicationEnd): JValue = { diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 09b604a95a309..fd6977fac3c4e 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -275,9 +275,11 @@ class JsonProtocolSuite extends FunSuite { test("SparkListenerApplicationStart backwards compatibility") { // SparkListenerApplicationStart in Spark 1.0.0 do not have an "appId" property. + // SparkListenerApplicationStart pre-Spark 1.4 does not have "appAttemptId". val applicationStart = SparkListenerApplicationStart("test", None, 1L, "user", None) val oldEvent = JsonProtocol.applicationStartToJson(applicationStart) .removeField({ _._1 == "App ID" }) + .removeField({ _._1 == "App Attempt ID" }) assert(applicationStart === JsonProtocol.applicationStartFromJson(oldEvent)) } From 9d59d92deff81863c8de58f8909bd0cf929b60c3 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 23 Apr 2015 11:43:55 -0700 Subject: [PATCH 17/22] Scalastyle... --- .../apache/spark/deploy/history/FsHistoryProviderSuite.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index d3e49938a17cd..245579f3806fd 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -67,7 +67,8 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers ) // Write a new-style application log. - val newAppCompressedComplete = newLogFile("new1compressed", None, inProgress = false, Some("lzf")) + val newAppCompressedComplete = newLogFile("new1compressed", None, inProgress = false, + Some("lzf")) writeFile(newAppCompressedComplete, true, None, SparkListenerApplicationStart("new-app-compressed-complete", None, 1L, "test", None), SparkListenerApplicationEnd(4L)) From 2ad77e7f93aec5cc0f9d1b0136352fe22bab5683 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 23 Apr 2015 14:00:13 -0700 Subject: [PATCH 18/22] Missed a reference to the old property name. --- .../main/scala/org/apache/spark/util/JsonProtocol.scala | 2 +- .../scala/org/apache/spark/util/JsonProtocolSuite.scala | 8 +++++--- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 39387d0642cba..8e0a327ae3599 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -563,7 +563,7 @@ private[spark] object JsonProtocol { val appId = Utils.jsonOption(json \ "App ID").map(_.extract[String]) val time = (json \ "Timestamp").extract[Long] val sparkUser = (json \ "User").extract[String] - val appAttemptId = Utils.jsonOption(json \ "appAttemptId").map(_.extract[String]) + val appAttemptId = Utils.jsonOption(json \ "App Attempt ID").map(_.extract[String]) SparkListenerApplicationStart(appName, appId, time, sparkUser, appAttemptId) } diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index fd6977fac3c4e..34282ae420d67 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -74,8 +74,8 @@ class JsonProtocolSuite extends FunSuite { val blockManagerRemoved = SparkListenerBlockManagerRemoved(2L, BlockManagerId("Scarce", "to be counted...", 100)) val unpersistRdd = SparkListenerUnpersistRDD(12345) - val applicationStart = SparkListenerApplicationStart("The winner of all", None, 42L, "Garfield", - None) + val applicationStart = SparkListenerApplicationStart("The winner of all", Some("appId"), + 42L, "Garfield", Some("appAttempt")) val applicationEnd = SparkListenerApplicationEnd(42L) val logUrlMap = Map("stderr" -> "mystderr", "stdout" -> "mystdout").toMap val executorAdded = SparkListenerExecutorAdded(executorAddedTime, "exec1", @@ -1500,8 +1500,10 @@ class JsonProtocolSuite extends FunSuite { |{ | "Event": "SparkListenerApplicationStart", | "App Name": "The winner of all", + | "App ID": "appId", | "Timestamp": 42, - | "User": "Garfield" + | "User": "Garfield", + | "App Attempt ID": "appAttempt" |} """ From 1aa309d4a3b29fc106629f035ce24c604853bf1c Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 23 Apr 2015 15:27:00 -0700 Subject: [PATCH 19/22] Improve sorting of app attempts. Make sure running attempts show up first. Also, when an app has running attempts, don't show it in the complete app list. --- .../deploy/history/FsHistoryProvider.scala | 28 +++++++++++++++---- .../spark/deploy/history/HistoryPage.scala | 2 +- .../history/FsHistoryProviderSuite.scala | 2 +- 3 files changed, 25 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index a69b7e9aad4a5..a777d2bdffef8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -270,7 +270,9 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis val newIterator = newApps.iterator.buffered val oldIterator = applications.values.iterator.buffered while (newIterator.hasNext && oldIterator.hasNext) { - if (compareAppInfo(newIterator.head, oldIterator.head)) { + if (newAppMap.contains(oldIterator.head.id)) { + oldIterator.next() + } else if (compareAppInfo(newIterator.head, oldIterator.head)) { addIfAbsent(newIterator.next()) } else { addIfAbsent(oldIterator.next()) @@ -328,13 +330,29 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis private def compareAppInfo( i1: FsApplicationHistoryInfo, i2: FsApplicationHistoryInfo): Boolean = { - compareAttemptInfo(i1.attempts.head, i2.attempts.head) + val a1 = i1.attempts.head + val a2 = i2.attempts.head + if (a1.endTime != a2.endTime) a1.endTime >= a2.endTime else a1.startTime >= a2.startTime } + /** + * Comparison function that defines the sort order for application attempts within the same + * application. Order is: running attempts before complete attempts, running attempts sorted + * by start time, completed attempts sorted by end time. + * + * Normally applications should have a single running attempt; but failure to call sc.stop() + * may cause multiple running attempts to show up. + * + * @return Whether `a1` should precede `a2`. + */ private def compareAttemptInfo( - i1: FsApplicationAttemptInfo, - i2: FsApplicationAttemptInfo): Boolean = { - if (i1.endTime != i2.endTime) i1.endTime >= i2.endTime else i1.startTime >= i2.startTime + a1: FsApplicationAttemptInfo, + a2: FsApplicationAttemptInfo): Boolean = { + if (a1.completed == a2.completed) { + if (a1.completed) a1.endTime >= a2.endTime else a1.startTime >= a2.startTime + } else { + !a1.completed + } } /** diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index f60babaf9af57..9595c89529280 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -38,7 +38,7 @@ private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") Option(request.getParameter("showIncomplete")).getOrElse("false").toBoolean val allApps = parent.getApplicationList() - .filter(_.attempts.exists(_.completed != requestedIncomplete)) + .filter(_.attempts.head.completed != requestedIncomplete) val allAppsSize = allApps.size val actualFirst = if (requestedFirst < allAppsSize) requestedFirst else 0 diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index 245579f3806fd..bfee97e1d3a21 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -244,7 +244,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers updateAndCheck(provider) { list => list.size should be (1) list.head.attempts.size should be (2) - list.head.attempts.head.attemptId should be ("attempt1") + list.head.attempts.head.attemptId should be ("attempt2") } val completedAttempt2 = newLogFile("app1", Some("attempt2"), inProgress = false) From 76a365182fffd9878304f49805e40553eba07b1e Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 24 Apr 2015 13:06:30 -0700 Subject: [PATCH 20/22] Fix log cleaner, add test. --- .../deploy/history/FsHistoryProvider.scala | 51 ++++++++++++------- .../history/FsHistoryProviderSuite.scala | 48 ++++++++++++++++- 2 files changed, 79 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 68b882857543b..a4d3308efae01 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -32,7 +32,7 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io.CompressionCodec import org.apache.spark.scheduler._ import org.apache.spark.ui.SparkUI -import org.apache.spark.util.{ThreadUtils, Utils} +import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils} import org.apache.spark.{Logging, SecurityManager, SparkConf} /** @@ -40,8 +40,12 @@ import org.apache.spark.{Logging, SecurityManager, SparkConf} * This provider checks for new finished applications in the background periodically and * renders the history application UI by parsing the associated event logs. */ -private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider - with Logging { +private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) + extends ApplicationHistoryProvider with Logging { + + def this(conf: SparkConf) = { + this(conf, new SystemClock()) + } import FsHistoryProvider._ @@ -75,8 +79,8 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis @volatile private var applications: mutable.LinkedHashMap[String, FsApplicationHistoryInfo] = new mutable.LinkedHashMap() - // List of applications to be deleted by event log cleaner. - private var appsToClean = new mutable.ListBuffer[FsApplicationHistoryInfo] + // List of application logs to be deleted by event log cleaner. + private var attemptsToClean = new mutable.ListBuffer[FsApplicationAttemptInfo] // Constants used to parse Spark 1.0.0 log directories. private[history] val LOG_PREFIX = "EVENT_LOG_" @@ -289,42 +293,51 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis /** * Delete event logs from the log directory according to the clean policy defined by the user. */ - private def cleanLogs(): Unit = { + private[history] def cleanLogs(): Unit = { try { val maxAge = conf.getTimeAsSeconds("spark.history.fs.cleaner.maxAge", "7d") * 1000 - val now = System.currentTimeMillis() + val now = clock.getTimeMillis() val appsToRetain = new mutable.LinkedHashMap[String, FsApplicationHistoryInfo]() + def shouldClean(attempt: FsApplicationAttemptInfo): Boolean = { + now - attempt.lastUpdated > maxAge && attempt.completed + } + // Scan all logs from the log directory. // Only completed applications older than the specified max age will be deleted. - applications.values.foreach { info => - if (now - info.attempts.head.lastUpdated <= maxAge || !info.attempts.head.completed) { - appsToRetain += (info.id -> info) - } else { - appsToClean += info + applications.values.foreach { app => + val toClean = app.attempts.filter(shouldClean) + attemptsToClean ++= toClean + + if (toClean.isEmpty) { + appsToRetain += (app.id -> app) + } else if (toClean.size < app.attempts.size) { + appsToRetain += (app.id -> + new FsApplicationHistoryInfo(app.id, app.name, + app.attempts.filter(!shouldClean(_)).toList)) } } applications = appsToRetain - val leftToClean = new mutable.ListBuffer[FsApplicationHistoryInfo] - appsToClean.foreach { info => + val leftToClean = new mutable.ListBuffer[FsApplicationAttemptInfo] + attemptsToClean.foreach { attempt => try { - val path = new Path(logDir, info.logPath) + val path = new Path(logDir, attempt.logPath) if (fs.exists(path)) { fs.delete(path, true) } } catch { case e: AccessControlException => - logInfo(s"No permission to delete ${info.logPath}, ignoring.") + logInfo(s"No permission to delete ${attempt.logPath}, ignoring.") case t: IOException => - logError(s"IOException in cleaning logs of ${info.logPath}", t) - leftToClean += info + logError(s"IOException in cleaning ${attempt.logPath}", t) + leftToClean += attempt } } - appsToClean = leftToClean + attemptsToClean = leftToClean } catch { case t: Exception => logError("Exception in cleaning logs", t) } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index bfee97e1d3a21..32a0e891f7026 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.deploy.history import java.io.{BufferedOutputStream, File, FileOutputStream, OutputStreamWriter} import java.net.URI +import java.util.concurrent.TimeUnit import scala.io.Source @@ -30,7 +31,7 @@ import org.scalatest.Matchers import org.apache.spark.{Logging, SparkConf} import org.apache.spark.io._ import org.apache.spark.scheduler._ -import org.apache.spark.util.{JsonProtocol, Utils} +import org.apache.spark.util.{JsonProtocol, ManualClock, Utils} class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers with Logging { @@ -283,6 +284,50 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers } } + test("log cleaner") { + val maxAge = TimeUnit.SECONDS.toMillis(10) + val clock = new ManualClock(maxAge / 2) + val provider = new FsHistoryProvider( + createTestConf().set("spark.history.fs.cleaner.maxAge", s"${maxAge}ms"), clock) + + val log1 = newLogFile("app1", Some("attempt1"), inProgress = false) + writeFile(log1, true, None, + SparkListenerApplicationStart("app1", Some("app1"), 1L, "test", Some("attempt1")), + SparkListenerApplicationEnd(2L) + ) + log1.setLastModified(0L) + + val log2 = newLogFile("app1", Some("attempt2"), inProgress = false) + writeFile(log2, true, None, + SparkListenerApplicationStart("app1", Some("app1"), 3L, "test", Some("attempt2")), + SparkListenerApplicationEnd(4L) + ) + log2.setLastModified(clock.getTimeMillis()) + + updateAndCheck(provider) { list => + list.size should be (1) + list.head.attempts.size should be (2) + } + + // Move the clock forward so log1 exceeds the max age. + clock.advance(maxAge) + + updateAndCheck(provider) { list => + list.size should be (1) + list.head.attempts.size should be (1) + list.head.attempts.head.attemptId should be ("attempt2") + } + assert(!log1.exists()) + + // Do the same for the other log. + clock.advance(maxAge) + + updateAndCheck(provider) { list => + list.size should be (0) + } + assert(!log2.exists()) + } + /** * Asks the provider to check for logs and calls a function to perform checks on the updated * app list. Example: @@ -294,6 +339,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers private def updateAndCheck(provider: FsHistoryProvider) (checkFn: Seq[ApplicationHistoryInfo] => Unit): Unit = { provider.checkForLogs() + provider.cleanLogs() checkFn(provider.getListing().toSeq) } From bc885b7295ade58f1e0c3159d4e5a2cd7658ef3b Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 28 Apr 2015 10:10:55 -0700 Subject: [PATCH 21/22] Review feedback. --- .../deploy/history/ApplicationHistoryProvider.scala | 7 +++---- .../spark/deploy/history/FsHistoryProvider.scala | 13 ++++++------- .../apache/spark/deploy/history/HistoryPage.scala | 12 ++++++------ .../apache/spark/deploy/history/HistoryServer.scala | 6 +++--- .../deploy/history/FsHistoryProviderSuite.scala | 13 +++++++------ .../spark/deploy/history/HistoryServerSuite.scala | 2 +- 6 files changed, 26 insertions(+), 27 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala index 9291d66e14cd4..6a5011af17458 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala @@ -20,7 +20,7 @@ package org.apache.spark.deploy.history import org.apache.spark.ui.SparkUI private[history] case class ApplicationAttemptInfo( - attemptId: String, + attemptId: Option[String], startTime: Long, endTime: Long, lastUpdated: Long, @@ -45,11 +45,10 @@ private[history] abstract class ApplicationHistoryProvider { * Returns the Spark UI for a specific application. * * @param appId The application ID. - * @param attemptId The application attempt ID for apps with multiple attempts (or an empty - * string for apps with a single attempt). + * @param attemptId The application attempt ID (or None if there is no attempt ID). * @return The application's UI, or None if application is not found. */ - def getAppUI(appId: String, attemptId: String): Option[SparkUI] + def getAppUI(appId: String, attemptId: Option[String]): Option[SparkUI] /** * Called when the server is shutting down. diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index a4d3308efae01..60918ec44360f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -142,7 +142,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) override def getListing(): Iterable[FsApplicationHistoryInfo] = applications.values - override def getAppUI(appId: String, attemptId: String): Option[SparkUI] = { + override def getAppUI(appId: String, attemptId: Option[String]): Option[SparkUI] = { try { applications.get(appId).flatMap { appInfo => val attempts = appInfo.attempts.filter(_.attemptId == attemptId) @@ -307,15 +307,14 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // Scan all logs from the log directory. // Only completed applications older than the specified max age will be deleted. applications.values.foreach { app => - val toClean = app.attempts.filter(shouldClean) + val (toClean, toRetain) = app.attempts.partition(shouldClean) attemptsToClean ++= toClean if (toClean.isEmpty) { appsToRetain += (app.id -> app) - } else if (toClean.size < app.attempts.size) { + } else if (toRetain.nonEmpty) { appsToRetain += (app.id -> - new FsApplicationHistoryInfo(app.id, app.name, - app.attempts.filter(!shouldClean(_)).toList)) + new FsApplicationHistoryInfo(app.id, app.name, toRetain.toList)) } } @@ -397,7 +396,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) logPath.getName(), appListener.appName.getOrElse(NOT_STARTED), appListener.appId.getOrElse(logPath.getName()), - appListener.appAttemptId.getOrElse(""), + appListener.appAttemptId, appListener.startTime.getOrElse(-1L), appListener.endTime.getOrElse(-1L), getModificationTime(eventLog).get, @@ -487,7 +486,7 @@ private class FsApplicationAttemptInfo( val logPath: String, val name: String, val appId: String, - attemptId: String, + attemptId: Option[String], startTime: Long, endTime: Long, lastUpdated: Long, diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index 9595c89529280..1af233f09f385 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -42,7 +42,7 @@ private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") val allAppsSize = allApps.size val actualFirst = if (requestedFirst < allAppsSize) requestedFirst else 0 - val appsToShow = allApps.slice(actualFirst, Math.min(actualFirst + pageSize, allAppsSize)) + val appsToShow = allApps.slice(actualFirst, actualFirst + pageSize) val actualPage = (actualFirst / pageSize) + 1 val last = Math.min(actualFirst + pageSize, allAppsSize) - 1 @@ -167,9 +167,9 @@ private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") val duration = if (attempt.endTime > 0) { UIUtils.formatDuration(attempt.endTime - attempt.startTime) - } else { - "-" - } + } else { + "-" + } val lastUpdated = UIUtils.formatDate(attempt.lastUpdated) { @@ -189,9 +189,9 @@ private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") } { if (renderAttemptIdColumn) { - if (info.attempts.size > 1 && !attempt.attemptId.isEmpty) { + if (info.attempts.size > 1 && attempt.attemptId.isDefined) { - {attempt.attemptId} + {attempt.attemptId.get} } else {   } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 1d97c6db4d007..f2883d4e2671b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -55,7 +55,7 @@ class HistoryServer( val parts = key.split("/") require(parts.length == 1 || parts.length == 2, s"Invalid app key $key") val ui = provider - .getAppUI(parts(0), if (parts.length > 1) parts(1) else "") + .getAppUI(parts(0), if (parts.length > 1) Some(parts(1)) else None) .getOrElse(throw new NoSuchElementException()) attachSparkUI(ui) ui @@ -222,8 +222,8 @@ object HistoryServer extends Logging { } } - private[history] def getAttemptURI(appId: String, attemptId: String): String = { - val attemptSuffix = if (!attemptId.isEmpty) s"/${attemptId}" else "" + private[history] def getAttemptURI(appId: String, attemptId: Option[String]): String = { + val attemptSuffix = attemptId.map { id => s"/$id" }.getOrElse("") s"${HistoryServer.UI_PATH_PREFIX}/${appId}${attemptSuffix}" } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index 32a0e891f7026..443b52380092e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -111,7 +111,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers def makeAppInfo(id: String, name: String, start: Long, end: Long, lastMod: Long, user: String, completed: Boolean): ApplicationHistoryInfo = { ApplicationHistoryInfo(id, name, - List(ApplicationAttemptInfo("", start, end, lastMod, user, completed))) + List(ApplicationAttemptInfo(None, start, end, lastMod, user, completed))) } list(0) should be (makeAppInfo(newAppComplete.getName(), "new-app-complete", 1L, 5L, @@ -128,8 +128,9 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers // Make sure the UI can be rendered. list.foreach { case info => - val appUi = provider.getAppUI(info.id, "") + val appUi = provider.getAppUI(info.id, None) appUi should not be null + appUi should not be None } } } @@ -245,7 +246,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers updateAndCheck(provider) { list => list.size should be (1) list.head.attempts.size should be (2) - list.head.attempts.head.attemptId should be ("attempt2") + list.head.attempts.head.attemptId should be (Some("attempt2")) } val completedAttempt2 = newLogFile("app1", Some("attempt2"), inProgress = false) @@ -259,7 +260,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers list should not be (null) list.size should be (1) list.head.attempts.size should be (2) - list.head.attempts.head.attemptId should be ("attempt2") + list.head.attempts.head.attemptId should be (Some("attempt2")) } val app2Attempt1 = newLogFile("app2", Some("attempt1"), inProgress = false) @@ -272,7 +273,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers list.size should be (2) list.head.attempts.size should be (1) list.last.attempts.size should be (2) - list.head.attempts.head.attemptId should be ("attempt1") + list.head.attempts.head.attemptId should be (Some("attempt1")) list.foreach { case app => app.attempts.foreach { attempt => @@ -315,7 +316,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers updateAndCheck(provider) { list => list.size should be (1) list.head.attempts.size should be (1) - list.head.attempts.head.attemptId should be ("attempt2") + list.head.attempts.head.attemptId should be (Some("attempt2")) } assert(!log1.exists()) diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index 420e382a29cbf..71ba9c18257b8 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -37,7 +37,7 @@ class HistoryServerSuite extends FunSuite with Matchers with MockitoSugar { val ui = mock[SparkUI] val link = "/history/app1" val info = new ApplicationHistoryInfo("app1", "app1", - List(ApplicationAttemptInfo("", 0, 2, 1, "xxx", true))) + List(ApplicationAttemptInfo(None, 0, 2, 1, "xxx", true))) when(historyServer.getApplicationList()).thenReturn(Seq(info)) when(ui.basePath).thenReturn(link) when(historyServer.getProviderConfig()).thenReturn(Map[String, String]()) From 7e289fa1297aa21f6c8764e4a237eb3a674675e0 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 30 Apr 2015 15:16:26 -0700 Subject: [PATCH 22/22] Review feedback. --- .../spark/deploy/history/FsHistoryProvider.scala | 3 +-- .../apache/spark/deploy/history/HistoryPage.scala | 3 --- .../apache/spark/deploy/history/HistoryServer.scala | 2 ++ .../spark/scheduler/EventLoggingListener.scala | 1 - .../apache/spark/scheduler/SchedulerBackend.scala | 5 +++-- .../deploy/history/FsHistoryProviderSuite.scala | 10 ++++++++-- .../apache/spark/deploy/yarn/ApplicationMaster.scala | 2 +- .../cluster/YarnClusterSchedulerBackend.scala | 12 +++++------- 8 files changed, 20 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 2096fc70ae235..993763f3aa092 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -145,8 +145,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) override def getAppUI(appId: String, attemptId: Option[String]): Option[SparkUI] = { try { applications.get(appId).flatMap { appInfo => - val attempts = appInfo.attempts.filter(_.attemptId == attemptId) - attempts.headOption.map { attempt => + appInfo.attempts.find(_.attemptId == attemptId).map { attempt => val replayBus = new ReplayListenerBus() val ui = { val conf = this.conf.clone() diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index 1af233f09f385..0830cc1ba1245 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -22,9 +22,6 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.ui.{WebUIPage, UIUtils} -import scala.collection.immutable.ListMap -import scala.collection.mutable.HashMap -import scala.collection.mutable.ArrayBuffer private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index f2883d4e2671b..754c8e9b6668b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -73,6 +73,8 @@ class HistoryServer( private val loaderServlet = new HttpServlet { protected override def doGet(req: HttpServletRequest, res: HttpServletResponse): Unit = { + // Parse the URI created by getAttemptURI(). It contains an app ID and an optional + // attempt ID (separated by a slash). val parts = Option(req.getPathInfo()).getOrElse("").split("/") if (parts.length < 2) { res.sendError(HttpServletResponse.SC_BAD_REQUEST, diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index eba50b54fc7d8..529a5b2bf1a0d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -269,7 +269,6 @@ private[spark] object EventLoggingListener extends Logging { appId: String, appAttemptId: Option[String], compressionCodecName: Option[String] = None): String = { - val sanitizedAppId = appId.replaceAll("[ :/]", "-").replaceAll("[.${}'\"]", "_").toLowerCase val base = logBaseDir.toString.stripSuffix("/") + "/" + sanitize(appId) val codec = compressionCodecName.map("." + _).getOrElse("") if (appAttemptId.isDefined) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala index 70500ccdb6212..646820520ea1b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala @@ -42,9 +42,10 @@ private[spark] trait SchedulerBackend { def applicationId(): String = appId /** - * Get an application ID associated with the job. + * Get the attempt ID for this run, if the cluster manager supports multiple + * attempts. Applications run in client mode will not have attempt IDs. * - * @return An application attempt id + * @return The application attempt id, if available. */ def applicationAttemptId(): Option[String] = None diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index 469ed19f8a252..a0a0afa48833e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -108,8 +108,14 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers list.size should be (5) list.count(_.attempts.head.completed) should be (3) - def makeAppInfo(id: String, name: String, start: Long, end: Long, lastMod: Long, - user: String, completed: Boolean): ApplicationHistoryInfo = { + def makeAppInfo( + id: String, + name: String, + start: Long, + end: Long, + lastMod: Long, + user: String, + completed: Boolean): ApplicationHistoryInfo = { ApplicationHistoryInfo(id, name, List(ApplicationAttemptInfo(None, start, end, lastMod, user, completed))) } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index e92e048712acf..27f804782f355 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -92,7 +92,7 @@ private[spark] class ApplicationMaster( // Propagate the attempt if, so that in case of event logging, // different attempt's logs gets created in different directory - System.setProperty("spark.yarn.app.attemptid", appAttemptId.getAttemptId().toString()) + System.setProperty("spark.yarn.app.attemptId", appAttemptId.getAttemptId().toString()) } logInfo("ApplicationAttemptId: " + appAttemptId) diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala index 28b212146553f..aeb218a575455 100644 --- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala @@ -39,19 +39,17 @@ private[spark] class YarnClusterSchedulerBackend( } override def applicationId(): String = - // In YARN Cluster mode, spark.yarn.app.id is expect to be set - // before user application is launched. - // So, if spark.yarn.app.id is not set, it is something wrong. + // In YARN Cluster mode, the application ID is expected to be set, so log an error if it's + // not found. sc.getConf.getOption("spark.yarn.app.id").getOrElse { logError("Application ID is not set.") super.applicationId } override def applicationAttemptId(): Option[String] = - // In YARN Cluster mode, spark.yarn.app.attemptid is expect to be set - // before user application is launched. - // So, if spark.yarn.app.id is not set, it is something wrong. - sc.getConf.getOption("spark.yarn.app.attemptid").orElse { + // In YARN Cluster mode, the attempt ID is expected to be set, so log an error if it's + // not found. + sc.getConf.getOption("spark.yarn.app.attemptId").orElse { logError("Application attempt ID is not set.") super.applicationAttemptId }