diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index c4579cf6ad560..ceead59b79ed6 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -17,8 +17,6 @@ package org.apache.spark -import scala.{Option, deprecated} - import org.apache.spark.util.collection.{AppendOnlyMap, ExternalAppendOnlyMap} /** diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index 872e892c04fe6..c7893f288b4b5 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -20,11 +20,12 @@ package org.apache.spark import scala.collection.mutable.{ArrayBuffer, HashSet} import org.apache.spark.rdd.RDD -import org.apache.spark.storage.{BlockManager, RDDBlockId, StorageLevel} +import org.apache.spark.storage.{BlockId, BlockManager, BlockStatus, RDDBlockId, StorageLevel} -/** Spark class responsible for passing RDDs split contents to the BlockManager and making - sure a node doesn't load two copies of an RDD at once. - */ +/** + * Spark class responsible for passing RDDs split contents to the BlockManager and making + * sure a node doesn't load two copies of an RDD at once. + */ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { /** Keys of RDD splits that are being computed/loaded. */ @@ -49,11 +50,11 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { try {loading.wait()} catch {case _ : Throwable =>} } logInfo("Finished waiting for %s".format(key)) - // See whether someone else has successfully loaded it. The main way this would fail - // is for the RDD-level cache eviction policy if someone else has loaded the same RDD - // partition but we didn't want to make space for it. However, that case is unlikely - // because it's unlikely that two threads would work on the same RDD partition. One - // downside of the current code is that threads wait serially if this does happen. + /* See whether someone else has successfully loaded it. The main way this would fail + * is for the RDD-level cache eviction policy if someone else has loaded the same RDD + * partition but we didn't want to make space for it. However, that case is unlikely + * because it's unlikely that two threads would work on the same RDD partition. One + * downside of the current code is that threads wait serially if this does happen. */ blockManager.get(key) match { case Some(values) => return new InterruptibleIterator(context, values.asInstanceOf[Iterator[T]]) @@ -69,32 +70,45 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { // If we got here, we have to load the split logInfo("Partition %s not found, computing it".format(key)) val computedValues = rdd.computeOrReadCheckpoint(split, context) + // Persist the result, so long as the task is not running locally if (context.runningLocally) { return computedValues } - if (storageLevel.useDisk && !storageLevel.useMemory) { - // In the case that this RDD is to be persisted using DISK_ONLY - // the iterator will be passed directly to the blockManager (rather then - // caching it to an ArrayBuffer first), then the resulting block data iterator - // will be passed back to the user. If the iterator generates a lot of data, - // this means that it doesn't all have to be held in memory at one time. - // This could also apply to MEMORY_ONLY_SER storage, but we need to make sure - // blocks aren't dropped by the block store before enabling that. - blockManager.put(key, computedValues, storageLevel, tellMaster = true) - return blockManager.get(key) match { - case Some(values) => - return new InterruptibleIterator(context, values.asInstanceOf[Iterator[T]]) - case None => - logInfo("Failure to store %s".format(key)) - throw new Exception("Block manager failed to return persisted valued") + + // Keep track of blocks with updated statuses + var updatedBlocks = Seq[(BlockId, BlockStatus)]() + val returnValue: Iterator[T] = { + if (storageLevel.useDisk && !storageLevel.useMemory) { + /* In the case that this RDD is to be persisted using DISK_ONLY + * the iterator will be passed directly to the blockManager (rather then + * caching it to an ArrayBuffer first), then the resulting block data iterator + * will be passed back to the user. If the iterator generates a lot of data, + * this means that it doesn't all have to be held in memory at one time. + * This could also apply to MEMORY_ONLY_SER storage, but we need to make sure + * blocks aren't dropped by the block store before enabling that. */ + updatedBlocks = blockManager.put(key, computedValues, storageLevel, tellMaster = true) + blockManager.get(key) match { + case Some(values) => + new InterruptibleIterator(context, values.asInstanceOf[Iterator[T]]) + case None => + logInfo("Failure to store %s".format(key)) + throw new Exception("Block manager failed to return persisted valued") + } + } else { + // In this case the RDD is cached to an array buffer. This will save the results + // if we're dealing with a 'one-time' iterator + val elements = new ArrayBuffer[Any] + elements ++= computedValues + updatedBlocks = blockManager.put(key, elements, storageLevel, tellMaster = true) + elements.iterator.asInstanceOf[Iterator[T]] } - } else { - // In this case the RDD is cached to an array buffer. This will save the results - // if we're dealing with a 'one-time' iterator - val elements = new ArrayBuffer[Any] - elements ++= computedValues - blockManager.put(key, elements, storageLevel, tellMaster = true) - return elements.iterator.asInstanceOf[Iterator[T]] } + + // Update task metrics to include any blocks whose storage status is updated + val metrics = context.taskMetrics + metrics.updatedBlocks = Some(updatedBlocks) + + returnValue + } finally { loading.synchronized { loading.remove(key) diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index 591978c1d3630..2237ee3bb7aad 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -18,13 +18,13 @@ package org.apache.spark import java.net.{Authenticator, PasswordAuthentication} -import org.apache.hadoop.io.Text -import org.apache.hadoop.security.Credentials -import org.apache.hadoop.security.UserGroupInformation -import org.apache.spark.deploy.SparkHadoopUtil import scala.collection.mutable.ArrayBuffer +import org.apache.hadoop.io.Text + +import org.apache.spark.deploy.SparkHadoopUtil + /** * Spark class responsible for security. * diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 852ed8fe1fb91..a1003b7925715 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -132,6 +132,9 @@ class SparkContext( if (master == "yarn-client") System.setProperty("SPARK_YARN_MODE", "true") + // An asynchronous listener bus for Spark events + private[spark] val listenerBus = new LiveListenerBus + // Create the Spark execution environment (cache, map output tracker, etc) private[spark] val env = SparkEnv.create( conf, @@ -139,7 +142,8 @@ class SparkContext( conf.get("spark.driver.host"), conf.get("spark.driver.port").toInt, isDriver = true, - isLocal = isLocal) + isLocal = isLocal, + listenerBus = listenerBus) SparkEnv.set(env) // Used to store a URL for each static file/jar together with the file's local timestamp @@ -151,9 +155,26 @@ class SparkContext( private[spark] val metadataCleaner = new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup, conf) - // Initialize the Spark UI + // Initialize the Spark UI, registering all associated listeners private[spark] val ui = new SparkUI(this) ui.bind() + ui.start() + + // Optionally log Spark events + private[spark] val eventLogger: Option[EventLoggingListener] = { + if (conf.getBoolean("spark.eventLog.enabled", false)) { + val logger = new EventLoggingListener(appName, conf) + listenerBus.addListener(logger) + Some(logger) + } else None + } + + // Information needed to replay logged events, if any + private[spark] val eventLoggingInfo: Option[EventLoggingInfo] = + eventLogger.map { logger => Some(logger.info) }.getOrElse(None) + + // At this point, all relevant SparkListeners have been registered, so begin releasing events + listenerBus.start() val startTime = System.currentTimeMillis() @@ -200,13 +221,13 @@ class SparkContext( executorEnvs("SPARK_USER") = sparkUser // Create and start the scheduler - private[spark] var taskScheduler = SparkContext.createTaskScheduler(this, master, appName) + private[spark] var taskScheduler = SparkContext.createTaskScheduler(this, master) taskScheduler.start() - @volatile private[spark] var dagScheduler = new DAGScheduler(taskScheduler) + @volatile private[spark] var dagScheduler = new DAGScheduler(this) dagScheduler.start() - ui.start() + postEnvironmentUpdate() /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ val hadoopConfiguration = { @@ -571,7 +592,6 @@ class SparkContext( .flatMap(x => Utils.deserialize[Array[T]](x._2.getBytes)) } - protected[spark] def checkpointFile[T: ClassTag]( path: String ): RDD[T] = { @@ -641,10 +661,11 @@ class SparkContext( Utils.fetchFile(path, new File(SparkFiles.getRootDirectory), conf, env.securityManager) logInfo("Added file " + path + " at " + key + " with timestamp " + addedFiles(key)) + postEnvironmentUpdate() } def addSparkListener(listener: SparkListener) { - dagScheduler.addSparkListener(listener) + listenerBus.addListener(listener) } /** @@ -671,7 +692,7 @@ class SparkContext( */ def getPersistentRDDs: Map[Int, RDD[_]] = persistentRdds.toMap - def getStageInfo: Map[Stage,StageInfo] = { + def getStageInfo: Map[Stage, StageInfo] = { dagScheduler.stageToInfos } @@ -698,7 +719,7 @@ class SparkContext( } /** - * Return current scheduling mode + * Return current scheduling mode */ def getSchedulingMode: SchedulingMode.SchedulingMode = { taskScheduler.schedulingMode @@ -708,6 +729,7 @@ class SparkContext( * Clear the job's list of files added by `addFile` so that they do not get downloaded to * any new nodes. */ + @deprecated("adding files no longer creates local copies that need to be deleted", "1.0.0") def clearFiles() { addedFiles.clear() } @@ -722,6 +744,23 @@ class SparkContext( dagScheduler.getPreferredLocs(rdd, partition) } + /** + * Register an RDD to be persisted in memory and/or disk storage + */ + private[spark] def persistRDD(rdd: RDD[_]) { + persistentRdds(rdd.id) = rdd + } + + /** + * Unpersist an RDD from memory and/or disk storage + */ + private[spark] def unpersistRDD(rdd: RDD[_], blocking: Boolean = true) { + val rddId = rdd.id + env.blockManager.master.removeRdd(rddId, blocking) + persistentRdds.remove(rddId) + listenerBus.post(SparkListenerUnpersistRDD(rddId)) + } + /** * Adds a JAR dependency for all tasks to be executed on this SparkContext in the future. * The `path` passed can be either a local file, a file in HDFS (or other Hadoop-supported @@ -744,7 +783,7 @@ class SparkContext( if (SparkHadoopUtil.get.isYarnMode() && (master == "yarn-standalone" || master == "yarn-cluster")) { // In order for this to work in yarn-cluster mode the user must specify the - // --addjars option to the client to upload the file into the distributed cache + // --addjars option to the client to upload the file into the distributed cache // of the AM to make it show up in the current working directory. val fileName = new Path(uri.getPath).getName() try { @@ -752,7 +791,7 @@ class SparkContext( } catch { case e: Exception => { // For now just log an error but allow to go through so spark examples work. - // The spark examples don't really need the jar distributed since its also + // The spark examples don't really need the jar distributed since its also // the app jar. logError("Error adding jar (" + e + "), was the --addJars option used?") null @@ -773,12 +812,14 @@ class SparkContext( logInfo("Added JAR " + path + " at " + key + " with timestamp " + addedJars(key)) } } + postEnvironmentUpdate() } /** * Clear the job's list of JARs added by `addJar` so that they do not get downloaded to * any new nodes. */ + @deprecated("adding jars no longer creates local copies that need to be deleted", "1.0.0") def clearJars() { addedJars.clear() } @@ -786,6 +827,7 @@ class SparkContext( /** Shut down the SparkContext. */ def stop() { ui.stop() + eventLogger.foreach(_.stop()) // Do this only if not stopped already - best case effort. // prevent NPE if stopped more than once. val dagSchedulerCopy = dagScheduler @@ -793,12 +835,10 @@ class SparkContext( if (dagSchedulerCopy != null) { metadataCleaner.cancel() dagSchedulerCopy.stop() + listenerBus.stop() taskScheduler = null // TODO: Cache.stop()? env.stop() - // Clean up locally linked files - clearFiles() - clearJars() SparkEnv.set(null) ShuffleMapTask.clearCache() ResultTask.clearCache() @@ -1026,6 +1066,19 @@ class SparkContext( /** Register a new RDD, returning its RDD ID */ private[spark] def newRddId(): Int = nextRddId.getAndIncrement() + /** Post the environment update event once the task scheduler is ready */ + private def postEnvironmentUpdate() { + if (taskScheduler != null) { + val schedulingMode = getSchedulingMode.toString + val addedJarPaths = addedJars.keys.toSeq + val addedFilePaths = addedFiles.keys.toSeq + val environmentDetails = + SparkEnv.environmentDetails(conf, schedulingMode, addedJarPaths, addedFilePaths) + val environmentUpdate = SparkListenerEnvironmentUpdate(environmentDetails) + listenerBus.post(environmentUpdate) + } + } + /** Called by MetadataCleaner to clean up the persistentRdds map periodically */ private[spark] def cleanup(cleanupTime: Long) { persistentRdds.clearOldValues(cleanupTime) @@ -1189,9 +1242,7 @@ object SparkContext extends Logging { } /** Creates a task scheduler based on a given master URL. Extracted for testing. */ - private def createTaskScheduler(sc: SparkContext, master: String, appName: String) - : TaskScheduler = - { + private def createTaskScheduler(sc: SparkContext, master: String): TaskScheduler = { // Regular expression used for local[N] master format val LOCAL_N_REGEX = """local\[([0-9]+)\]""".r // Regular expression for local[N, maxRetries], used in tests with failing tasks @@ -1230,7 +1281,7 @@ object SparkContext extends Logging { case SPARK_REGEX(sparkUrl) => val scheduler = new TaskSchedulerImpl(sc) val masterUrls = sparkUrl.split(",").map("spark://" + _) - val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls, appName) + val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls) scheduler.initialize(backend) scheduler @@ -1247,7 +1298,7 @@ object SparkContext extends Logging { val localCluster = new LocalSparkCluster( numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt) val masterUrls = localCluster.start() - val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls, appName) + val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls) scheduler.initialize(backend) backend.shutdownCallback = (backend: SparkDeploySchedulerBackend) => { localCluster.stop() @@ -1307,9 +1358,9 @@ object SparkContext extends Logging { val coarseGrained = sc.conf.getBoolean("spark.mesos.coarse", false) val url = mesosUrl.stripPrefix("mesos://") // strip scheme from raw Mesos URLs val backend = if (coarseGrained) { - new CoarseMesosSchedulerBackend(scheduler, sc, url, appName) + new CoarseMesosSchedulerBackend(scheduler, sc, url) } else { - new MesosSchedulerBackend(scheduler, sc, url, appName) + new MesosSchedulerBackend(scheduler, sc, url) } scheduler.initialize(backend) scheduler diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 774cbd6441a48..a1af63fa4a391 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -17,8 +17,10 @@ package org.apache.spark +import scala.collection.JavaConversions._ import scala.collection.mutable import scala.concurrent.Await +import scala.util.Properties import akka.actor._ import com.google.common.collect.MapMaker @@ -26,9 +28,10 @@ import com.google.common.collect.MapMaker import org.apache.spark.api.python.PythonWorkerFactory import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.metrics.MetricsSystem -import org.apache.spark.storage.{BlockManager, BlockManagerMaster, BlockManagerMasterActor} import org.apache.spark.network.ConnectionManager +import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.Serializer +import org.apache.spark.storage._ import org.apache.spark.util.{AkkaUtils, Utils} /** @@ -49,11 +52,11 @@ class SparkEnv private[spark] ( val broadcastManager: BroadcastManager, val blockManager: BlockManager, val connectionManager: ConnectionManager, + val securityManager: SecurityManager, val httpFileServer: HttpFileServer, val sparkFilesDir: String, val metricsSystem: MetricsSystem, - val conf: SparkConf, - val securityManager: SecurityManager) extends Logging { + val conf: SparkConf) extends Logging { // A mapping of thread ID to amount of memory used for shuffle in bytes // All accesses should be manually synchronized @@ -120,9 +123,16 @@ object SparkEnv extends Logging { hostname: String, port: Int, isDriver: Boolean, - isLocal: Boolean): SparkEnv = { + isLocal: Boolean, + listenerBus: LiveListenerBus = null): SparkEnv = { + + // Listener bus is only used on the driver + if (isDriver) { + assert(listenerBus != null, "Attempted to create driver SparkEnv with null listener bus!") + } val securityManager = new SecurityManager(conf) + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, port, conf = conf, securityManager = securityManager) @@ -172,8 +182,9 @@ object SparkEnv extends Logging { val blockManagerMaster = new BlockManagerMaster(registerOrLookup( "BlockManagerMaster", - new BlockManagerMasterActor(isLocal, conf)), conf) - val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, + new BlockManagerMasterActor(isLocal, conf, listenerBus)), conf) + + val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, serializer, conf, securityManager) val connectionManager = blockManager.connectionManager @@ -233,10 +244,63 @@ object SparkEnv extends Logging { broadcastManager, blockManager, connectionManager, + securityManager, httpFileServer, sparkFilesDir, metricsSystem, - conf, - securityManager) + conf) + } + + /** + * Return a map representation of jvm information, Spark properties, system properties, and + * class paths. Map keys define the category, and map values represent the corresponding + * attributes as a sequence of KV pairs. This is used mainly for SparkListenerEnvironmentUpdate. + */ + private[spark] + def environmentDetails( + conf: SparkConf, + schedulingMode: String, + addedJars: Seq[String], + addedFiles: Seq[String]): Map[String, Seq[(String, String)]] = { + + val jvmInformation = Seq( + ("Java Version", "%s (%s)".format(Properties.javaVersion, Properties.javaVendor)), + ("Java Home", Properties.javaHome), + ("Scala Version", Properties.versionString), + ("Scala Home", Properties.scalaHome) + ).sorted + + // Spark properties + // This includes the scheduling mode whether or not it is configured (used by SparkUI) + val schedulerMode = + if (!conf.contains("spark.scheduler.mode")) { + Seq(("spark.scheduler.mode", schedulingMode)) + } else { + Seq[(String, String)]() + } + val sparkProperties = (conf.getAll ++ schedulerMode).sorted + + // System properties that are not java classpaths + val systemProperties = System.getProperties.iterator.toSeq + val otherProperties = systemProperties.filter { case (k, v) => + k != "java.class.path" && !k.startsWith("spark.") + }.sorted + + // Class paths including all added jars and files + val classPathProperty = systemProperties.find { case (k, v) => + k == "java.class.path" + }.getOrElse(("", "")) + val classPathEntries = classPathProperty._2 + .split(conf.get("path.separator", ":")) + .filterNot(e => e.isEmpty) + .map(e => (e, "System Classpath")) + val addedJarsAndFiles = (addedJars ++ addedFiles).map((_, "Added By User")) + val classPaths = (addedJarsAndFiles ++ classPathEntries).sorted + + Map[String, Seq[(String, String)]]( + "JVM Information" -> jvmInformation, + "Spark Properties" -> sparkProperties, + "System Properties" -> otherProperties, + "Classpath Entries" -> classPaths) } } diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 3fd6f5eb472f4..f1a753b6ab8a9 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -29,7 +29,7 @@ private[spark] sealed trait TaskEndReason private[spark] case object Success extends TaskEndReason -private[spark] +private[spark] case object Resubmitted extends TaskEndReason // Task was finished earlier but we've now lost it private[spark] case class FetchFailed( @@ -65,4 +65,3 @@ private[spark] case object ExecutorLostFailure extends TaskEndReason * deserializing the task result. */ private[spark] case object UnknownReason extends TaskEndReason - diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 8e0eab56a3dcf..35508b6e5acba 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -434,6 +434,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * Clear the job's list of JARs added by `addJar` so that they do not get downloaded to * any new nodes. */ + @deprecated("adding jars no longer creates local copies that need to be deleted", "1.0.0") def clearJars() { sc.clearJars() } @@ -442,6 +443,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * Clear the job's list of files added by `addFile` so that they do not get downloaded to * any new nodes. */ + @deprecated("adding files no longer creates local copies that need to be deleted", "1.0.0") def clearFiles() { sc.clearFiles() } diff --git a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala index 449b953530ff9..15fa8a7679874 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala @@ -17,13 +17,16 @@ package org.apache.spark.deploy +import org.apache.spark.scheduler.EventLoggingInfo + private[spark] class ApplicationDescription( val name: String, val maxCores: Option[Int], val memoryPerSlave: Int, val command: Command, val sparkHome: Option[String], - val appUiUrl: String) + var appUiUrl: String, + val eventLogInfo: Option[EventLoggingInfo] = None) extends Serializable { val user = System.getProperty("user.name", "") diff --git a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala index cefb1ff97e83c..c4f5e294a393e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala @@ -43,7 +43,6 @@ private[spark] object JsonProtocol { ("starttime" -> obj.startTime) ~ ("id" -> obj.id) ~ ("name" -> obj.desc.name) ~ - ("appuiurl" -> obj.appUiUrl) ~ ("cores" -> obj.desc.maxCores) ~ ("user" -> obj.desc.user) ~ ("memoryperslave" -> obj.desc.memoryPerSlave) ~ diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala index e8867bc1691d3..46b9f4dc7d3ba 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala @@ -31,7 +31,6 @@ private[spark] class ApplicationInfo( val desc: ApplicationDescription, val submitDate: Date, val driver: ActorRef, - val appUiUrl: String, defaultCores: Int) extends Serializable { @@ -45,11 +44,6 @@ private[spark] class ApplicationInfo( init() - private def readObject(in: java.io.ObjectInputStream) : Unit = { - in.defaultReadObject() - init() - } - private def init() { state = ApplicationState.WAITING executors = new mutable.HashMap[Int, ExecutorInfo] 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 b8dfa44102583..1fd211416976e 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 @@ -37,10 +37,16 @@ import org.apache.spark.deploy.master.DriverState.DriverState import org.apache.spark.deploy.master.MasterMessages._ import org.apache.spark.deploy.master.ui.MasterWebUI import org.apache.spark.metrics.MetricsSystem +import org.apache.spark.scheduler.ReplayListenerBus +import org.apache.spark.ui.SparkUI import org.apache.spark.util.{AkkaUtils, Utils} -private[spark] class Master(host: String, port: Int, webUiPort: Int, +private[spark] class Master( + host: String, + port: Int, + webUiPort: Int, val securityMgr: SecurityManager) extends Actor with Logging { + import context.dispatcher // to use Akka's scheduler.schedule() val conf = new SparkConf @@ -64,6 +70,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int, val completedApps = new ArrayBuffer[ApplicationInfo] var nextAppNumber = 0 + val appIdToUI = new HashMap[String, SparkUI] + val drivers = new HashSet[DriverInfo] val completedDrivers = new ArrayBuffer[DriverInfo] val waitingDrivers = new ArrayBuffer[DriverInfo] // Drivers currently spooled for scheduling @@ -107,8 +115,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int, logInfo("Starting Spark master at " + masterUrl) // Listen for remote client disconnection events, since they don't go through Akka's watch() context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) - webUi.start() - masterWebUiUrl = "http://" + masterPublicAddress + ":" + webUi.boundPort.get + webUi.bind() + masterWebUiUrl = "http://" + masterPublicAddress + ":" + webUi.boundPort context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis, self, CheckForWorkerTimeOut) masterMetricsSystem.registerSource(masterSource) @@ -141,6 +149,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int, override def postStop() { webUi.stop() + appIdToUI.values.foreach(_.stop()) masterMetricsSystem.stop() applicationMetricsSystem.stop() persistenceEngine.close() @@ -373,7 +382,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int, } case RequestWebUIPort => { - sender ! WebUIPortResponse(webUi.boundPort.getOrElse(-1)) + sender ! WebUIPortResponse(webUi.boundPort) } } @@ -581,8 +590,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int, def createApplication(desc: ApplicationDescription, driver: ActorRef): ApplicationInfo = { val now = System.currentTimeMillis() val date = new Date(now) - new ApplicationInfo( - now, newApplicationId(date), desc, date, driver, desc.appUiUrl, defaultCores) + new ApplicationInfo(now, newApplicationId(date), desc, date, driver, defaultCores) } def registerApplication(app: ApplicationInfo): Unit = { @@ -614,12 +622,27 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int, if (completedApps.size >= RETAINED_APPLICATIONS) { val toRemove = math.max(RETAINED_APPLICATIONS / 10, 1) completedApps.take(toRemove).foreach( a => { + appIdToUI.remove(a.id).foreach { ui => + ui.stop() + webUi.detachUI(ui) + } applicationMetricsSystem.removeSource(a.appSource) }) completedApps.trimStart(toRemove) } completedApps += app // Remember it in our history waitingApps -= app + + // If application events are logged, use them to rebuild the UI + startPersistedSparkUI(app).map { ui => + app.desc.appUiUrl = ui.basePath + appIdToUI(app.id) = ui + webUi.attachUI(ui) + }.getOrElse { + // Avoid broken links if the UI is not reconstructed + app.desc.appUiUrl = "" + } + for (exec <- app.executors.values) { exec.worker.removeExecutor(exec) exec.worker.actor ! KillExecutor(masterUrl, exec.application.id, exec.id) @@ -634,6 +657,36 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int, } } + /** + * Start a new SparkUI rendered from persisted storage. If this is unsuccessful for any reason, + * return None. Otherwise return the reconstructed UI. + */ + def startPersistedSparkUI(app: ApplicationInfo): Option[SparkUI] = { + val appName = app.desc.name + val eventLogInfo = app.desc.eventLogInfo.getOrElse { return None } + val eventLogDir = eventLogInfo.logDir + val eventCompressionCodec = eventLogInfo.compressionCodec + val appConf = new SparkConf + eventCompressionCodec.foreach { codec => + appConf.set("spark.eventLog.compress", "true") + appConf.set("spark.io.compression.codec", codec) + } + val replayerBus = new ReplayListenerBus(appConf) + val ui = new SparkUI( + appConf, + replayerBus, + "%s (finished)".format(appName), + "/history/%s".format(app.id)) + + // Do not call ui.bind() to avoid creating a new server for each application + ui.start() + val success = replayerBus.replay(eventLogDir) + if (!success) { + ui.stop() + None + } else Some(ui) + } + /** Generate a new app ID given a app's submission date */ def newApplicationId(submitDate: Date): String = { val appId = "app-%s-%04d".format(DATE_FORMAT.format(submitDate), nextAppNumber) @@ -717,9 +770,11 @@ private[spark] object Master { } } - def startSystemAndActor(host: String, port: Int, webUiPort: Int, conf: SparkConf) - : (ActorSystem, Int, Int) = - { + def startSystemAndActor( + host: String, + port: Int, + webUiPort: Int, + conf: SparkConf): (ActorSystem, Int, Int) = { val securityMgr = new SecurityManager(conf) val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port, conf = conf, securityManager = securityMgr) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index 90cad3c37fda6..cb092cb5d576b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -23,7 +23,6 @@ import scala.concurrent.Await import scala.xml.Node import akka.pattern.ask -import javax.servlet.http.HttpServletRequest import org.json4s.JValue import org.apache.spark.deploy.JsonProtocol @@ -83,7 +82,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) {
  • Submit Date: {app.submitDate}
  • State: {app.state}
  • -
  • Application Detail UI
  • +
  • Application Detail UI
  • diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala index 3233cd97f7bd0..7ec71eb80bfc0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala @@ -23,7 +23,6 @@ import scala.concurrent.Await import scala.xml.Node import akka.pattern.ask -import javax.servlet.http.HttpServletRequest import org.json4s.JValue import org.apache.spark.deploy.{DeployWebUI, JsonProtocol} @@ -162,7 +161,7 @@ private[spark] class IndexPage(parent: MasterWebUI) { {app.id} - {app.desc.name} + {app.desc.name} {app.coresGranted} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index 4ad1f95be31c9..bd75b2dfd0e07 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -18,12 +18,12 @@ package org.apache.spark.deploy.master.ui import javax.servlet.http.HttpServletRequest -import org.eclipse.jetty.server.Server + import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.Logging import org.apache.spark.deploy.master.Master -import org.apache.spark.ui.JettyUtils +import org.apache.spark.ui.{ServerInfo, SparkUI} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.{AkkaUtils, Utils} @@ -32,24 +32,35 @@ import org.apache.spark.util.{AkkaUtils, Utils} */ private[spark] class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { - val timeout = AkkaUtils.askTimeout(master.conf) - val host = Utils.localHostName() - val port = requestedPort - val masterActorRef = master.self + val timeout = AkkaUtils.askTimeout(master.conf) - var server: Option[Server] = None - var boundPort: Option[Int] = None + private val host = Utils.localHostName() + private val port = requestedPort + private val applicationPage = new ApplicationPage(this) + private val indexPage = new IndexPage(this) + private var serverInfo: Option[ServerInfo] = None - val applicationPage = new ApplicationPage(this) - val indexPage = new IndexPage(this) + private val handlers: Seq[ServletContextHandler] = { + master.masterMetricsSystem.getServletHandlers ++ + master.applicationMetricsSystem.getServletHandlers ++ + Seq[ServletContextHandler]( + createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR, "/static"), + createServletHandler("/app/json", + (request: HttpServletRequest) => applicationPage.renderJson(request), master.securityMgr), + createServletHandler("/app", + (request: HttpServletRequest) => applicationPage.render(request), master.securityMgr), + createServletHandler("/json", + (request: HttpServletRequest) => indexPage.renderJson(request), master.securityMgr), + createServletHandler("/", + (request: HttpServletRequest) => indexPage.render(request), master.securityMgr) + ) + } - def start() { + def bind() { try { - val (srv, bPort) = JettyUtils.startJettyServer(host, port, handlers, master.conf) - server = Some(srv) - boundPort = Some(bPort) - logInfo("Started Master web UI at http://%s:%d".format(host, boundPort.get)) + serverInfo = Some(startJettyServer(host, port, handlers, master.conf)) + logInfo("Started Master web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => logError("Failed to create Master JettyUtils", e) @@ -57,27 +68,38 @@ class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { } } - val metricsHandlers = master.masterMetricsSystem.getServletHandlers ++ - master.applicationMetricsSystem.getServletHandlers + def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) - val handlers = metricsHandlers ++ Seq[ServletContextHandler]( - createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR + "/static", "/static"), - createServletHandler("/app/json", - createServlet((request: HttpServletRequest) => applicationPage.renderJson(request), - master.securityMgr)), - createServletHandler("/app", createServlet((request: HttpServletRequest) => applicationPage - .render(request), master.securityMgr)), - createServletHandler("/json", createServlet((request: HttpServletRequest) => indexPage - .renderJson(request), master.securityMgr)), - createServletHandler("*", createServlet((request: HttpServletRequest) => indexPage.render - (request), master.securityMgr)) - ) + /** Attach a reconstructed UI to this Master UI. Only valid after bind(). */ + def attachUI(ui: SparkUI) { + assert(serverInfo.isDefined, "Master UI must be bound to a server before attaching SparkUIs") + val rootHandler = serverInfo.get.rootHandler + for (handler <- ui.handlers) { + rootHandler.addHandler(handler) + if (!handler.isStarted) { + handler.start() + } + } + } + + /** Detach a reconstructed UI from this Master UI. Only valid after bind(). */ + def detachUI(ui: SparkUI) { + assert(serverInfo.isDefined, "Master UI must be bound to a server before detaching SparkUIs") + val rootHandler = serverInfo.get.rootHandler + for (handler <- ui.handlers) { + if (handler.isStarted) { + handler.stop() + } + rootHandler.removeHandler(handler) + } + } def stop() { - server.foreach(_.stop()) + assert(serverInfo.isDefined, "Attempted to stop a Master UI that was not bound to a server!") + serverInfo.get.server.stop() } } private[spark] object MasterWebUI { - val STATIC_RESOURCE_DIR = "org/apache/spark/ui" + val STATIC_RESOURCE_DIR = SparkUI.STATIC_RESOURCE_DIR } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index afaabedffefea..5e0fc31fff22f 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -124,7 +124,7 @@ private[spark] class Worker( createWorkDir() webUi = new WorkerWebUI(this, workDir, Some(webUiPort)) context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) - webUi.start() + webUi.bind() registerWithMaster() metricsSystem.registerSource(workerSource) @@ -150,8 +150,7 @@ private[spark] class Worker( for (masterUrl <- masterUrls) { logInfo("Connecting to master " + masterUrl + "...") val actor = context.actorSelection(Master.toAkkaUrl(masterUrl)) - actor ! RegisterWorker(workerId, host, port, cores, memory, webUi.boundPort.get, - publicAddress) + actor ! RegisterWorker(workerId, host, port, cores, memory, webUi.boundPort, publicAddress) } } @@ -340,10 +339,15 @@ private[spark] object Worker { actorSystem.awaitTermination() } - def startSystemAndActor(host: String, port: Int, webUiPort: Int, cores: Int, memory: Int, - masterUrls: Array[String], workDir: String, workerNumber: Option[Int] = None) - : (ActorSystem, Int) = - { + def startSystemAndActor( + host: String, + port: Int, + webUiPort: Int, + cores: Int, + memory: Int, + masterUrls: Array[String], + workDir: String, workerNumber: Option[Int] = None): (ActorSystem, Int) = { + // The LocalSparkCluster runs multiple local sparkWorkerX actor systems val conf = new SparkConf val systemName = "sparkWorker" + workerNumber.map(_.toString).getOrElse("") diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index 4e33b330ad4e7..de76a5d5eb7bc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -19,12 +19,12 @@ package org.apache.spark.deploy.worker.ui import java.io.File import javax.servlet.http.HttpServletRequest -import org.eclipse.jetty.server.Server + import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.Logging import org.apache.spark.deploy.worker.Worker -import org.apache.spark.ui.{JettyUtils, UIUtils} +import org.apache.spark.ui.{JettyUtils, ServerInfo, SparkUI, UIUtils} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.{AkkaUtils, Utils} @@ -33,37 +33,35 @@ import org.apache.spark.util.{AkkaUtils, Utils} */ private[spark] class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[Int] = None) - extends Logging { - val timeout = AkkaUtils.askTimeout(worker.conf) - val host = Utils.localHostName() - val port = requestedPort.getOrElse( - worker.conf.get("worker.ui.port", WorkerWebUI.DEFAULT_PORT).toInt) - - var server: Option[Server] = None - var boundPort: Option[Int] = None - - val indexPage = new IndexPage(this) + extends Logging { - val metricsHandlers = worker.metricsSystem.getServletHandlers + val timeout = AkkaUtils.askTimeout(worker.conf) - val handlers = metricsHandlers ++ Seq[ServletContextHandler]( - createStaticHandler(WorkerWebUI.STATIC_RESOURCE_BASE + "/static", "/static"), - createServletHandler("/log", createServlet((request: HttpServletRequest) => log(request), - worker.securityMgr)), - createServletHandler("/logPage", createServlet((request: HttpServletRequest) => logPage - (request), worker.securityMgr)), - createServletHandler("/json", createServlet((request: HttpServletRequest) => indexPage - .renderJson(request), worker.securityMgr)), - createServletHandler("*", createServlet((request: HttpServletRequest) => indexPage.render - (request), worker.securityMgr)) - ) + private val host = Utils.localHostName() + private val port = requestedPort.getOrElse( + worker.conf.get("worker.ui.port", WorkerWebUI.DEFAULT_PORT).toInt) + private val indexPage = new IndexPage(this) + private var serverInfo: Option[ServerInfo] = None + + private val handlers: Seq[ServletContextHandler] = { + worker.metricsSystem.getServletHandlers ++ + Seq[ServletContextHandler]( + createStaticHandler(WorkerWebUI.STATIC_RESOURCE_BASE, "/static"), + createServletHandler("/log", + (request: HttpServletRequest) => log(request), worker.securityMgr), + createServletHandler("/logPage", + (request: HttpServletRequest) => logPage(request), worker.securityMgr), + createServletHandler("/json", + (request: HttpServletRequest) => indexPage.renderJson(request), worker.securityMgr), + createServletHandler("/", + (request: HttpServletRequest) => indexPage.render(request), worker.securityMgr) + ) + } - def start() { + def bind() { try { - val (srv, bPort) = JettyUtils.startJettyServer(host, port, handlers, worker.conf) - server = Some(srv) - boundPort = Some(bPort) - logInfo("Started Worker web UI at http://%s:%d".format(host, bPort)) + serverInfo = Some(JettyUtils.startJettyServer(host, port, handlers, worker.conf)) + logInfo("Started Worker web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => logError("Failed to create Worker JettyUtils", e) @@ -71,7 +69,9 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I } } - def log(request: HttpServletRequest): String = { + def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) + + private def log(request: HttpServletRequest): String = { val defaultBytes = 100 * 1024 val appId = Option(request.getParameter("appId")) @@ -98,7 +98,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I pre + Utils.offsetBytes(path, startByte, endByte) } - def logPage(request: HttpServletRequest): Seq[scala.xml.Node] = { + private def logPage(request: HttpServletRequest): Seq[scala.xml.Node] = { val defaultBytes = 100 * 1024 val appId = Option(request.getParameter("appId")) val executorId = Option(request.getParameter("executorId")) @@ -119,17 +119,14 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val (startByte, endByte) = getByteRange(path, offset, byteLength) val file = new File(path) val logLength = file.length - val logText = {Utils.offsetBytes(path, startByte, endByte)} - val linkToMaster =

    Back to Master

    - val range = Bytes {startByte.toString} - {endByte.toString} of {logLength} val backButton = if (startByte > 0) { + .format(params, logType, math.max(startByte - byteLength, 0), byteLength)}> @@ -146,7 +143,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I } @@ -175,33 +172,28 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I } /** Determine the byte range for a log or log page. */ - def getByteRange(path: String, offset: Option[Long], byteLength: Int) - : (Long, Long) = { + private def getByteRange(path: String, offset: Option[Long], byteLength: Int): (Long, Long) = { val defaultBytes = 100 * 1024 val maxBytes = 1024 * 1024 - val file = new File(path) val logLength = file.length() - val getOffset = offset.getOrElse(logLength-defaultBytes) - + val getOffset = offset.getOrElse(logLength - defaultBytes) val startByte = if (getOffset < 0) 0L else if (getOffset > logLength) logLength else getOffset - val logPageLength = math.min(byteLength, maxBytes) - val endByte = math.min(startByte + logPageLength, logLength) - (startByte, endByte) } def stop() { - server.foreach(_.stop()) + assert(serverInfo.isDefined, "Attempted to stop a Worker UI that was not bound to a server!") + serverInfo.get.server.stop() } } private[spark] object WorkerWebUI { - val STATIC_RESOURCE_BASE = "org/apache/spark/ui" + val STATIC_RESOURCE_BASE = SparkUI.STATIC_RESOURCE_DIR val DEFAULT_PORT="8081" } diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 2ea2ec29f59f5..8fe9b848ba145 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -224,10 +224,10 @@ private[spark] class Executor( for (m <- task.metrics) { m.hostname = Utils.localHostName() - m.executorDeserializeTime = (taskStart - startTime).toInt - m.executorRunTime = (taskFinish - taskStart).toInt + m.executorDeserializeTime = taskStart - startTime + m.executorRunTime = taskFinish - taskStart m.jvmGCTime = gcTime - startGCTime - m.resultSerializationTime = (afterSerialization - beforeSerialization).toInt + m.resultSerializationTime = afterSerialization - beforeSerialization } val accumUpdates = Accumulators.values @@ -263,7 +263,7 @@ private[spark] class Executor( } case t: Throwable => { - val serviceTime = (System.currentTimeMillis() - taskStart).toInt + val serviceTime = System.currentTimeMillis() - taskStart val metrics = attemptedTask.flatMap(t => t.metrics) for (m <- metrics) { m.executorRunTime = serviceTime diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 760458cb02a9b..88625e79a5c68 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -17,21 +17,23 @@ package org.apache.spark.executor +import org.apache.spark.storage.{BlockId, BlockStatus} + class TaskMetrics extends Serializable { /** - * Host's name the task runs on + * Host's name the task runs on */ var hostname: String = _ /** * Time taken on the executor to deserialize this task */ - var executorDeserializeTime: Int = _ + var executorDeserializeTime: Long = _ /** * Time the executor spends actually running the task (including fetching shuffle data) */ - var executorRunTime: Int = _ + var executorRunTime: Long = _ /** * The number of bytes this task transmitted back to the driver as the TaskResult @@ -68,6 +70,11 @@ class TaskMetrics extends Serializable { * here */ var shuffleWriteMetrics: Option[ShuffleWriteMetrics] = None + + /** + * Storage statuses of any blocks that have been updated as a result of this task. + */ + var updatedBlocks: Option[Seq[(BlockId, BlockStatus)]] = None } object TaskMetrics { diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala index 848b5c439bb5b..059e58824c39b 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -38,8 +38,7 @@ trait CompressionCodec { private[spark] object CompressionCodec { def createCodec(conf: SparkConf): CompressionCodec = { - createCodec(conf, conf.get( - "spark.io.compression.codec", classOf[LZFCompressionCodec].getName)) + createCodec(conf, conf.get("spark.io.compression.codec", DEFAULT_COMPRESSION_CODEC)) } def createCodec(conf: SparkConf, codecName: String): CompressionCodec = { @@ -47,6 +46,8 @@ private[spark] object CompressionCodec { .getConstructor(classOf[SparkConf]) ctor.newInstance(conf).asInstanceOf[CompressionCodec] } + + val DEFAULT_COMPRESSION_CODEC = classOf[LZFCompressionCodec].getName } diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala index 3110eccdee4fc..854b52c510e3d 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala @@ -28,7 +28,7 @@ import com.fasterxml.jackson.databind.ObjectMapper import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.SecurityManager -import org.apache.spark.ui.JettyUtils +import org.apache.spark.ui.JettyUtils._ class MetricsServlet(val property: Properties, val registry: MetricRegistry, securityMgr: SecurityManager) extends Sink { @@ -46,10 +46,8 @@ class MetricsServlet(val property: Properties, val registry: MetricRegistry, new MetricsModule(TimeUnit.SECONDS, TimeUnit.MILLISECONDS, servletShowSample)) def getHandlers = Array[ServletContextHandler]( - JettyUtils.createServletHandler(servletPath, - JettyUtils.createServlet( - new JettyUtils.ServletParams(request => getMetricsSnapshot(request), "text/json"), - securityMgr) ) + createServletHandler(servletPath, + new ServletParams(request => getMetricsSnapshot(request), "text/json"), securityMgr) ) def getMetricsSnapshot(request: HttpServletRequest): String = { diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index ddb901246d360..1b43040c6d918 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -137,9 +137,8 @@ abstract class RDD[T: ClassTag]( throw new UnsupportedOperationException( "Cannot change storage level of an RDD after it was already assigned a level") } + sc.persistRDD(this) storageLevel = newLevel - // Register the RDD with the SparkContext - sc.persistentRdds(id) = this this } @@ -157,8 +156,7 @@ abstract class RDD[T: ClassTag]( */ def unpersist(blocking: Boolean = true): RDD[T] = { logInfo("Removing RDD " + id + " from persistence list") - sc.env.blockManager.master.removeRdd(id, blocking) - sc.persistentRdds.remove(id) + sc.unpersistRDD(this, blocking) storageLevel = StorageLevel.NONE this } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index d83d0341c61ab..77c558ac46f6f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -32,7 +32,7 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} import org.apache.spark.rdd.RDD import org.apache.spark.storage.{BlockId, BlockManager, BlockManagerMaster, RDDBlockId} -import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashMap} +import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashMap, Utils} /** * The high-level scheduling layer that implements stage-oriented scheduling. It computes a DAG of @@ -54,87 +54,53 @@ import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedH */ private[spark] class DAGScheduler( - taskSched: TaskScheduler, + taskScheduler: TaskScheduler, + listenerBus: LiveListenerBus, mapOutputTracker: MapOutputTrackerMaster, blockManagerMaster: BlockManagerMaster, env: SparkEnv) extends Logging { - def this(taskSched: TaskScheduler) { - this(taskSched, SparkEnv.get.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster], - SparkEnv.get.blockManager.master, SparkEnv.get) - } - taskSched.setDAGScheduler(this) + import DAGScheduler._ - // Called by TaskScheduler to report task's starting. - def taskStarted(task: Task[_], taskInfo: TaskInfo) { - eventProcessActor ! BeginEvent(task, taskInfo) - } - - // Called to report that a task has completed and results are being fetched remotely. - def taskGettingResult(task: Task[_], taskInfo: TaskInfo) { - eventProcessActor ! GettingResultEvent(task, taskInfo) + def this(sc: SparkContext, taskScheduler: TaskScheduler) = { + this( + taskScheduler, + sc.listenerBus, + sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster], + sc.env.blockManager.master, + sc.env) } - // Called by TaskScheduler to report task completions or failures. - def taskEnded( - task: Task[_], - reason: TaskEndReason, - result: Any, - accumUpdates: Map[Long, Any], - taskInfo: TaskInfo, - taskMetrics: TaskMetrics) { - eventProcessActor ! CompletionEvent(task, reason, result, accumUpdates, taskInfo, taskMetrics) - } - - // Called by TaskScheduler when an executor fails. - def executorLost(execId: String) { - eventProcessActor ! ExecutorLost(execId) - } - - // Called by TaskScheduler when a host is added - def executorGained(execId: String, host: String) { - eventProcessActor ! ExecutorGained(execId, host) - } - - // Called by TaskScheduler to cancel an entire TaskSet due to either repeated failures or - // cancellation of the job itself. - def taskSetFailed(taskSet: TaskSet, reason: String) { - eventProcessActor ! TaskSetFailed(taskSet, reason) - } - - // The time, in millis, to wait for fetch failure events to stop coming in after one is detected; - // this is a simplistic way to avoid resubmitting tasks in the non-fetchable map stage one by one - // as more failure events come in - val RESUBMIT_TIMEOUT = 200.milliseconds - - // The time, in millis, to wake up between polls of the completion queue in order to potentially - // resubmit failed stages - val POLL_TIMEOUT = 10L - - // Warns the user if a stage contains a task with size greater than this value (in KB) - val TASK_SIZE_TO_WARN = 100 + def this(sc: SparkContext) = this(sc, sc.taskScheduler) private var eventProcessActor: ActorRef = _ private[scheduler] val nextJobId = new AtomicInteger(0) - - def numTotalJobs: Int = nextJobId.get() - + private[scheduler] def numTotalJobs: Int = nextJobId.get() private val nextStageId = new AtomicInteger(0) private[scheduler] val jobIdToStageIds = new TimeStampedHashMap[Int, HashSet[Int]] - private[scheduler] val stageIdToJobIds = new TimeStampedHashMap[Int, HashSet[Int]] - private[scheduler] val stageIdToStage = new TimeStampedHashMap[Int, Stage] - private[scheduler] val shuffleToMapStage = new TimeStampedHashMap[Int, Stage] - + private[scheduler] val stageIdToActiveJob = new HashMap[Int, ActiveJob] + private[scheduler] val resultStageToJob = new HashMap[Stage, ActiveJob] private[spark] val stageToInfos = new TimeStampedHashMap[Stage, StageInfo] - // An async scheduler event bus. The bus should be stopped when DAGSCheduler is stopped. - private[spark] val listenerBus = new SparkListenerBus + // Stages we need to run whose parents aren't done + private[scheduler] val waitingStages = new HashSet[Stage] + + // Stages we are running right now + private[scheduler] val runningStages = new HashSet[Stage] + + // Stages that must be resubmitted due to fetch failures + private[scheduler] val failedStages = new HashSet[Stage] + + // Missing tasks from each stage + private[scheduler] val pendingTasks = new TimeStampedHashMap[Stage, HashSet[Task[_]]] + + private[scheduler] val activeJobs = new HashSet[ActiveJob] // Contains the locations that each RDD's partitions are cached on private val cacheLocs = new HashMap[Int, Array[Seq[TaskLocation]]] @@ -145,22 +111,12 @@ class DAGScheduler( // // TODO: Garbage collect information about failure epochs when we know there are no more // stray messages to detect. - val failedEpoch = new HashMap[String, Long] + private val failedEpoch = new HashMap[String, Long] - // stage id to the active job - val idToActiveJob = new HashMap[Int, ActiveJob] + private val metadataCleaner = + new MetadataCleaner(MetadataCleanerType.DAG_SCHEDULER, this.cleanup, env.conf) - val waiting = new HashSet[Stage] // Stages we need to run whose parents aren't done - val running = new HashSet[Stage] // Stages we are running right now - val failed = new HashSet[Stage] // Stages that must be resubmitted due to fetch failures - // Missing tasks from each stage - val pendingTasks = new TimeStampedHashMap[Stage, HashSet[Task[_]]] - - val activeJobs = new HashSet[ActiveJob] - val resultStageToJob = new HashMap[Stage, ActiveJob] - - val metadataCleaner = new MetadataCleaner( - MetadataCleanerType.DAG_SCHEDULER, this.cleanup, env.conf) + taskScheduler.setDAGScheduler(this) /** * Starts the event processing actor. The actor has two responsibilities: @@ -196,13 +152,46 @@ class DAGScheduler( })) } - def addSparkListener(listener: SparkListener) { - listenerBus.addListener(listener) + // Called by TaskScheduler to report task's starting. + def taskStarted(task: Task[_], taskInfo: TaskInfo) { + eventProcessActor ! BeginEvent(task, taskInfo) + } + + // Called to report that a task has completed and results are being fetched remotely. + def taskGettingResult(task: Task[_], taskInfo: TaskInfo) { + eventProcessActor ! GettingResultEvent(task, taskInfo) + } + + // Called by TaskScheduler to report task completions or failures. + def taskEnded( + task: Task[_], + reason: TaskEndReason, + result: Any, + accumUpdates: Map[Long, Any], + taskInfo: TaskInfo, + taskMetrics: TaskMetrics) { + eventProcessActor ! CompletionEvent(task, reason, result, accumUpdates, taskInfo, taskMetrics) + } + + // Called by TaskScheduler when an executor fails. + def executorLost(execId: String) { + eventProcessActor ! ExecutorLost(execId) + } + + // Called by TaskScheduler when a host is added + def executorAdded(execId: String, host: String) { + eventProcessActor ! ExecutorAdded(execId, host) + } + + // Called by TaskScheduler to cancel an entire TaskSet due to either repeated failures or + // cancellation of the job itself. + def taskSetFailed(taskSet: TaskSet, reason: String) { + eventProcessActor ! TaskSetFailed(taskSet, reason) } private def getCacheLocs(rdd: RDD[_]): Array[Seq[TaskLocation]] = { if (!cacheLocs.contains(rdd.id)) { - val blockIds = rdd.partitions.indices.map(index=> RDDBlockId(rdd.id, index)).toArray[BlockId] + val blockIds = rdd.partitions.indices.map(index => RDDBlockId(rdd.id, index)).toArray[BlockId] val locs = BlockManager.blockIdsToBlockManagers(blockIds, env, blockManagerMaster) cacheLocs(rdd.id) = blockIds.map { id => locs.getOrElse(id, Nil).map(bm => TaskLocation(bm.host, bm.executorId)) @@ -250,7 +239,7 @@ class DAGScheduler( new Stage(id, rdd, numTasks, shuffleDep, getParentStages(rdd, jobId), jobId, callSite) stageIdToStage(id) = stage updateJobIdStageIdMaps(jobId, stage) - stageToInfos(stage) = new StageInfo(stage) + stageToInfos(stage) = StageInfo.fromStage(stage) stage } @@ -376,9 +365,9 @@ class DAGScheduler( def removeStage(stageId: Int) { // data structures based on Stage for (stage <- stageIdToStage.get(stageId)) { - if (running.contains(stage)) { + if (runningStages.contains(stage)) { logDebug("Removing running stage %d".format(stageId)) - running -= stage + runningStages -= stage } stageToInfos -= stage for ((k, v) <- shuffleToMapStage.find(_._2 == stage)) { @@ -388,13 +377,13 @@ class DAGScheduler( logDebug("Removing pending status for stage %d".format(stageId)) } pendingTasks -= stage - if (waiting.contains(stage)) { + if (waitingStages.contains(stage)) { logDebug("Removing stage %d from waiting set.".format(stageId)) - waiting -= stage + waitingStages -= stage } - if (failed.contains(stage)) { + if (failedStages.contains(stage)) { logDebug("Removing stage %d from failed set.".format(stageId)) - failed -= stage + failedStages -= stage } } // data structures based on StageId @@ -544,13 +533,14 @@ class DAGScheduler( logInfo("Missing parents: " + getMissingParentStages(finalStage)) if (allowLocal && finalStage.parents.size == 0 && partitions.length == 1) { // Compute very short actions like first() or take() with no parent stages locally. - listenerBus.post(SparkListenerJobStart(job, Array(), properties)) + listenerBus.post(SparkListenerJobStart(job.jobId, Array[Int](), properties)) runLocally(job) } else { - idToActiveJob(jobId) = job + stageIdToActiveJob(jobId) = job activeJobs += job resultStageToJob(finalStage) = job - listenerBus.post(SparkListenerJobStart(job, jobIdToStageIds(jobId).toArray, properties)) + listenerBus.post( + SparkListenerJobStart(job.jobId, jobIdToStageIds(jobId).toArray, properties)) submitStage(finalStage) } @@ -563,23 +553,23 @@ class DAGScheduler( val activeInGroup = activeJobs.filter(activeJob => groupId == activeJob.properties.get(SparkContext.SPARK_JOB_GROUP_ID)) val jobIds = activeInGroup.map(_.jobId) - jobIds.foreach { handleJobCancellation } + jobIds.foreach(handleJobCancellation) case AllJobsCancelled => // Cancel all running jobs. - running.map(_.jobId).foreach { handleJobCancellation } + runningStages.map(_.jobId).foreach(handleJobCancellation) activeJobs.clear() // These should already be empty by this point, - idToActiveJob.clear() // but just in case we lost track of some jobs... + stageIdToActiveJob.clear() // but just in case we lost track of some jobs... - case ExecutorGained(execId, host) => - handleExecutorGained(execId, host) + case ExecutorAdded(execId, host) => + handleExecutorAdded(execId, host) case ExecutorLost(execId) => handleExecutorLost(execId) case BeginEvent(task, taskInfo) => for ( - job <- idToActiveJob.get(task.stageId); + job <- stageIdToActiveJob.get(task.stageId); stage <- stageIdToStage.get(task.stageId); stageInfo <- stageToInfos.get(stage) ) { @@ -591,20 +581,22 @@ class DAGScheduler( task.stageId, stageInfo.name, taskInfo.serializedSize / 1024, TASK_SIZE_TO_WARN)) } } - listenerBus.post(SparkListenerTaskStart(task, taskInfo)) + listenerBus.post(SparkListenerTaskStart(task.stageId, taskInfo)) case GettingResultEvent(task, taskInfo) => - listenerBus.post(SparkListenerTaskGettingResult(task, taskInfo)) + listenerBus.post(SparkListenerTaskGettingResult(taskInfo)) case completion @ CompletionEvent(task, reason, _, _, taskInfo, taskMetrics) => - listenerBus.post(SparkListenerTaskEnd(task, reason, taskInfo, taskMetrics)) + val stageId = task.stageId + val taskType = Utils.getFormattedClassName(task) + listenerBus.post(SparkListenerTaskEnd(stageId, taskType, reason, taskInfo, taskMetrics)) handleTaskCompletion(completion) case TaskSetFailed(taskSet, reason) => stageIdToStage.get(taskSet.stageId).foreach { abortStage(_, reason) } case ResubmitFailedStages => - if (failed.size > 0) { + if (failedStages.size > 0) { // Failed stages may be removed by job cancellation, so failed might be empty even if // the ResubmitFailedStages event has been scheduled. resubmitFailedStages() @@ -615,7 +607,7 @@ class DAGScheduler( for (job <- activeJobs) { val error = new SparkException("Job cancelled because SparkContext was shut down") job.listener.jobFailed(error) - listenerBus.post(SparkListenerJobEnd(job, JobFailed(error, None))) + listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error, -1))) } return true } @@ -629,9 +621,9 @@ class DAGScheduler( private[scheduler] def resubmitFailedStages() { logInfo("Resubmitting failed stages") clearCacheLocs() - val failed2 = failed.toArray - failed.clear() - for (stage <- failed2.sortBy(_.jobId)) { + val failedStagesCopy = failedStages.toArray + failedStages.clear() + for (stage <- failedStagesCopy.sortBy(_.jobId)) { submitStage(stage) } } @@ -644,12 +636,12 @@ class DAGScheduler( // TODO: We might want to run this less often, when we are sure that something has become // runnable that wasn't before. logTrace("Checking for newly runnable parent stages") - logTrace("running: " + running) - logTrace("waiting: " + waiting) - logTrace("failed: " + failed) - val waiting2 = waiting.toArray - waiting.clear() - for (stage <- waiting2.sortBy(_.jobId)) { + logTrace("running: " + runningStages) + logTrace("waiting: " + waitingStages) + logTrace("failed: " + failedStages) + val waitingStagesCopy = waitingStages.toArray + waitingStages.clear() + for (stage <- waitingStagesCopy.sortBy(_.jobId)) { submitStage(stage) } } @@ -685,7 +677,7 @@ class DAGScheduler( } } catch { case e: Exception => - jobResult = JobFailed(e, Some(job.finalStage)) + jobResult = JobFailed(e, job.finalStage.id) job.listener.jobFailed(e) } finally { val s = job.finalStage @@ -693,7 +685,7 @@ class DAGScheduler( stageIdToStage -= s.id // but that won't get cleaned up via the normal paths through stageToInfos -= s // completion events or stage abort jobIdToStageIds -= job.jobId - listenerBus.post(SparkListenerJobEnd(job, jobResult)) + listenerBus.post(SparkListenerJobEnd(job.jobId, jobResult)) } } @@ -705,7 +697,7 @@ class DAGScheduler( private def activeJobForStage(stage: Stage): Option[Int] = { if (stageIdToJobIds.contains(stage.id)) { val jobsThatUseStage: Array[Int] = stageIdToJobIds(stage.id).toArray.sorted - jobsThatUseStage.find(idToActiveJob.contains(_)) + jobsThatUseStage.find(stageIdToActiveJob.contains) } else { None } @@ -716,18 +708,18 @@ class DAGScheduler( val jobId = activeJobForStage(stage) if (jobId.isDefined) { logDebug("submitStage(" + stage + ")") - if (!waiting(stage) && !running(stage) && !failed(stage)) { + if (!waitingStages(stage) && !runningStages(stage) && !failedStages(stage)) { val missing = getMissingParentStages(stage).sortBy(_.id) logDebug("missing: " + missing) if (missing == Nil) { logInfo("Submitting " + stage + " (" + stage.rdd + "), which has no missing parents") submitMissingTasks(stage, jobId.get) - running += stage + runningStages += stage } else { for (parent <- missing) { submitStage(parent) } - waiting += stage + waitingStages += stage } } } else { @@ -758,8 +750,8 @@ class DAGScheduler( } } - val properties = if (idToActiveJob.contains(jobId)) { - idToActiveJob(stage.jobId).properties + val properties = if (stageIdToActiveJob.contains(jobId)) { + stageIdToActiveJob(stage.jobId).properties } else { //this stage will be assigned to "default" pool null @@ -779,20 +771,20 @@ class DAGScheduler( } catch { case e: NotSerializableException => abortStage(stage, "Task not serializable: " + e.toString) - running -= stage + runningStages -= stage return } logInfo("Submitting " + tasks.size + " missing tasks from " + stage + " (" + stage.rdd + ")") myPending ++= tasks logDebug("New pending tasks: " + myPending) - taskSched.submitTasks( + taskScheduler.submitTasks( new TaskSet(tasks.toArray, stage.id, stage.newAttemptId(), stage.jobId, properties)) stageToInfos(stage).submissionTime = Some(System.currentTimeMillis()) } else { logDebug("Stage " + stage + " is actually done; %b %d %d".format( stage.isAvailable, stage.numAvailableOutputs, stage.numPartitions)) - running -= stage + runningStages -= stage } } @@ -817,7 +809,7 @@ class DAGScheduler( logInfo("%s (%s) finished in %s s".format(stage, stage.name, serviceTime)) stageToInfos(stage).completionTime = Some(System.currentTimeMillis()) listenerBus.post(SparkListenerStageCompleted(stageToInfos(stage))) - running -= stage + runningStages -= stage } event.reason match { case Success => @@ -826,7 +818,6 @@ class DAGScheduler( Accumulators.add(event.accumUpdates) // TODO: do this only if task wasn't resubmitted } pendingTasks(stage) -= task - stageToInfos(stage).taskInfos += event.taskInfo -> event.taskMetrics task match { case rt: ResultTask[_, _] => resultStageToJob.get(stage) match { @@ -836,12 +827,12 @@ class DAGScheduler( job.numFinished += 1 // If the whole job has finished, remove it if (job.numFinished == job.numPartitions) { - idToActiveJob -= stage.jobId + stageIdToActiveJob -= stage.jobId activeJobs -= job resultStageToJob -= stage markStageAsFinished(stage) jobIdToStageIdsRemove(job.jobId) - listenerBus.post(SparkListenerJobEnd(job, JobSucceeded)) + listenerBus.post(SparkListenerJobEnd(job.jobId, JobSucceeded)) } job.listener.taskSucceeded(rt.outputId, event.result) } @@ -858,12 +849,12 @@ class DAGScheduler( } else { stage.addOutputLoc(smt.partitionId, status) } - if (running.contains(stage) && pendingTasks(stage).isEmpty) { + if (runningStages.contains(stage) && pendingTasks(stage).isEmpty) { markStageAsFinished(stage) logInfo("looking for newly runnable stages") - logInfo("running: " + running) - logInfo("waiting: " + waiting) - logInfo("failed: " + failed) + logInfo("running: " + runningStages) + logInfo("waiting: " + waitingStages) + logInfo("failed: " + failedStages) if (stage.shuffleDep.isDefined) { // We supply true to increment the epoch number here in case this is a // recomputation of the map outputs. In that case, some nodes may have cached @@ -886,14 +877,14 @@ class DAGScheduler( submitStage(stage) } else { val newlyRunnable = new ArrayBuffer[Stage] - for (stage <- waiting) { + for (stage <- waitingStages) { logInfo("Missing parents for " + stage + ": " + getMissingParentStages(stage)) } - for (stage <- waiting if getMissingParentStages(stage) == Nil) { + for (stage <- waitingStages if getMissingParentStages(stage) == Nil) { newlyRunnable += stage } - waiting --= newlyRunnable - running ++= newlyRunnable + waitingStages --= newlyRunnable + runningStages ++= newlyRunnable for { stage <- newlyRunnable.sortBy(_.id) jobId <- activeJobForStage(stage) @@ -912,7 +903,7 @@ class DAGScheduler( case FetchFailed(bmAddress, shuffleId, mapId, reduceId) => // Mark the stage that the reducer was in as unrunnable val failedStage = stageIdToStage(task.stageId) - running -= failedStage + runningStages -= failedStage // TODO: Cancel running tasks in the stage logInfo("Marking " + failedStage + " (" + failedStage.name + ") for resubmision due to a fetch failure") @@ -924,7 +915,7 @@ class DAGScheduler( } logInfo("The failed fetch was from " + mapStage + " (" + mapStage.name + "); marking it for resubmission") - if (failed.isEmpty && eventProcessActor != null) { + if (failedStages.isEmpty && eventProcessActor != null) { // Don't schedule an event to resubmit failed stages if failed isn't empty, because // in that case the event will already have been scheduled. eventProcessActor may be // null during unit tests. @@ -932,8 +923,8 @@ class DAGScheduler( env.actorSystem.scheduler.scheduleOnce( RESUBMIT_TIMEOUT, eventProcessActor, ResubmitFailedStages) } - failed += failedStage - failed += mapStage + failedStages += failedStage + failedStages += mapStage // TODO: mark the executor as failed only if there were lots of fetch failures on it if (bmAddress != null) { handleExecutorLost(bmAddress.executorId, Some(task.epoch)) @@ -980,10 +971,10 @@ class DAGScheduler( } } - private def handleExecutorGained(execId: String, host: String) { + private def handleExecutorAdded(execId: String, host: String) { // remove from failedEpoch(execId) ? if (failedEpoch.contains(execId)) { - logInfo("Host gained which was in lost list earlier: " + host) + logInfo("Host added was in lost list earlier: " + host) failedEpoch -= execId } } @@ -993,14 +984,14 @@ class DAGScheduler( logDebug("Trying to cancel unregistered job " + jobId) } else { val independentStages = removeJobAndIndependentStages(jobId) - independentStages.foreach { taskSched.cancelTasks } + independentStages.foreach(taskScheduler.cancelTasks) val error = new SparkException("Job %d cancelled".format(jobId)) - val job = idToActiveJob(jobId) + val job = stageIdToActiveJob(jobId) job.listener.jobFailed(error) jobIdToStageIds -= jobId activeJobs -= job - idToActiveJob -= jobId - listenerBus.post(SparkListenerJobEnd(job, JobFailed(error, Some(job.finalStage)))) + stageIdToActiveJob -= jobId + listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error, job.finalStage.id))) } } @@ -1020,10 +1011,10 @@ class DAGScheduler( val error = new SparkException("Job aborted: " + reason) job.listener.jobFailed(error) jobIdToStageIdsRemove(job.jobId) - idToActiveJob -= resultStage.jobId + stageIdToActiveJob -= resultStage.jobId activeJobs -= job resultStageToJob -= resultStage - listenerBus.post(SparkListenerJobEnd(job, JobFailed(error, Some(failedStage)))) + listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error, failedStage.id))) } if (dependentStages.isEmpty) { logInfo("Ignoring failure of " + failedStage + " because all jobs depending on it are done") @@ -1102,11 +1093,11 @@ class DAGScheduler( "stageToInfos" -> stageToInfos, "jobIdToStageIds" -> jobIdToStageIds, "stageIdToJobIds" -> stageIdToJobIds). - foreach { case(s, t) => { - val sizeBefore = t.size - t.clearOldValues(cleanupTime) - logInfo("%s %d --> %d".format(s, sizeBefore, t.size)) - }} + foreach { case (s, t) => + val sizeBefore = t.size + t.clearOldValues(cleanupTime) + logInfo("%s %d --> %d".format(s, sizeBefore, t.size)) + } } def stop() { @@ -1114,7 +1105,20 @@ class DAGScheduler( eventProcessActor ! StopDAGScheduler } metadataCleaner.cancel() - taskSched.stop() - listenerBus.stop() + taskScheduler.stop() } } + +private[spark] object DAGScheduler { + // The time, in millis, to wait for fetch failure events to stop coming in after one is detected; + // this is a simplistic way to avoid resubmitting tasks in the non-fetchable map stage one by one + // as more failure events come in + val RESUBMIT_TIMEOUT = 200.milliseconds + + // The time, in millis, to wake up between polls of the completion queue in order to potentially + // resubmit failed stages + val POLL_TIMEOUT = 10L + + // Warns the user if a stage contains a task with size greater than this value (in KB) + val TASK_SIZE_TO_WARN = 100 +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index 39cd98e2d74e4..04c53d468465a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -65,7 +65,7 @@ private[scheduler] case class CompletionEvent( taskMetrics: TaskMetrics) extends DAGSchedulerEvent -private[scheduler] case class ExecutorGained(execId: String, host: String) extends DAGSchedulerEvent +private[scheduler] case class ExecutorAdded(execId: String, host: String) extends DAGSchedulerEvent private[scheduler] case class ExecutorLost(execId: String) extends DAGSchedulerEvent diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala index b52fe2410abde..5878e733908f5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala @@ -28,15 +28,15 @@ private[spark] class DAGSchedulerSource(val dagScheduler: DAGScheduler, sc: Spar val sourceName = "%s.DAGScheduler".format(sc.appName) metricRegistry.register(MetricRegistry.name("stage", "failedStages"), new Gauge[Int] { - override def getValue: Int = dagScheduler.failed.size + override def getValue: Int = dagScheduler.failedStages.size }) metricRegistry.register(MetricRegistry.name("stage", "runningStages"), new Gauge[Int] { - override def getValue: Int = dagScheduler.running.size + override def getValue: Int = dagScheduler.runningStages.size }) metricRegistry.register(MetricRegistry.name("stage", "waitingStages"), new Gauge[Int] { - override def getValue: Int = dagScheduler.waiting.size + override def getValue: Int = dagScheduler.waitingStages.size }) metricRegistry.register(MetricRegistry.name("job", "allJobs"), new Gauge[Int] { diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala new file mode 100644 index 0000000000000..217f8825c2ae9 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.io.CompressionCodec +import org.apache.spark.util.{JsonProtocol, FileLogger} + +/** + * A SparkListener that logs events to persistent storage. + * + * Event logging is specified by the following configurable parameters: + * spark.eventLog.enabled - Whether event logging is enabled. + * spark.eventLog.compress - Whether to compress logged events + * spark.eventLog.overwrite - Whether to overwrite any existing files. + * spark.eventLog.dir - Path to the directory in which events are logged. + * spark.eventLog.buffer.kb - Buffer size to use when writing to output streams + */ +private[spark] class EventLoggingListener(appName: String, conf: SparkConf) + extends SparkListener with Logging { + + private val shouldCompress = conf.getBoolean("spark.eventLog.compress", false) + private val shouldOverwrite = conf.getBoolean("spark.eventLog.overwrite", false) + private val outputBufferSize = conf.getInt("spark.eventLog.buffer.kb", 100) * 1024 + private val logBaseDir = conf.get("spark.eventLog.dir", "/tmp/spark-events").stripSuffix("/") + private val name = appName.replaceAll("[ :/]", "-").toLowerCase + "-" + System.currentTimeMillis + val logDir = logBaseDir + "/" + name + + private val logger = + new FileLogger(logDir, conf, outputBufferSize, shouldCompress, shouldOverwrite) + + // Information needed to replay the events logged by this listener later + val info = { + val compressionCodec = if (shouldCompress) { + Some(conf.get("spark.io.compression.codec", CompressionCodec.DEFAULT_COMPRESSION_CODEC)) + } else None + EventLoggingInfo(logDir, compressionCodec) + } + + logInfo("Logging events to %s".format(logDir)) + + /** Log the event as JSON */ + private def logEvent(event: SparkListenerEvent, flushLogger: Boolean = false) { + val eventJson = compact(render(JsonProtocol.sparkEventToJson(event))) + logger.logLine(eventJson) + if (flushLogger) { + logger.flush() + } + } + + // Events that do not trigger a flush + override def onStageSubmitted(event: SparkListenerStageSubmitted) = + logEvent(event) + override def onTaskStart(event: SparkListenerTaskStart) = + logEvent(event) + override def onTaskGettingResult(event: SparkListenerTaskGettingResult) = + logEvent(event) + override def onTaskEnd(event: SparkListenerTaskEnd) = + logEvent(event) + override def onEnvironmentUpdate(event: SparkListenerEnvironmentUpdate) = + logEvent(event) + + // Events that trigger a flush + override def onStageCompleted(event: SparkListenerStageCompleted) = + logEvent(event, flushLogger = true) + override def onJobStart(event: SparkListenerJobStart) = + logEvent(event, flushLogger = true) + override def onJobEnd(event: SparkListenerJobEnd) = + logEvent(event, flushLogger = true) + override def onBlockManagerAdded(event: SparkListenerBlockManagerAdded) = + logEvent(event, flushLogger = true) + override def onBlockManagerRemoved(event: SparkListenerBlockManagerRemoved) = + logEvent(event, flushLogger = true) + override def onUnpersistRDD(event: SparkListenerUnpersistRDD) = + logEvent(event, flushLogger = true) + + def stop() = logger.stop() +} + +// If compression is not enabled, compressionCodec is None +private[spark] case class EventLoggingInfo(logDir: String, compressionCodec: Option[String]) diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index 01cbcc390c6cd..b3a67d7e17976 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -22,24 +22,25 @@ import java.text.SimpleDateFormat import java.util.{Date, Properties} import java.util.concurrent.LinkedBlockingQueue -import scala.collection.mutable.{HashMap, HashSet, ListBuffer} +import scala.collection.mutable.HashMap import org.apache.spark._ import org.apache.spark.executor.TaskMetrics -import org.apache.spark.rdd.RDD -import org.apache.spark.storage.StorageLevel /** * A logger class to record runtime information for jobs in Spark. This class outputs one log file - * for each Spark job, containing RDD graph, tasks start/stop, shuffle information. - * JobLogger is a subclass of SparkListener, use addSparkListener to add JobLogger to a SparkContext - * after the SparkContext is created. - * Note that each JobLogger only works for one SparkContext - * @param logDirName The base directory for the log files. + * for each Spark job, containing tasks start/stop and shuffle information. JobLogger is a subclass + * of SparkListener, use addSparkListener to add JobLogger to a SparkContext after the SparkContext + * is created. Note that each JobLogger only works for one SparkContext + * + * NOTE: The functionality of this class is heavily stripped down to accommodate for a general + * refactor of the SparkListener interface. In its place, the EventLoggingListener is introduced + * to log application information as SparkListenerEvents. To enable this functionality, set + * spark.eventLog.enabled to true. */ -class JobLogger(val user: String, val logDirName: String) - extends SparkListener with Logging { +@deprecated("Log application information by setting spark.eventLog.enabled.", "1.0.0") +class JobLogger(val user: String, val logDirName: String) extends SparkListener with Logging { def this() = this(System.getProperty("user.name", ""), String.valueOf(System.currentTimeMillis())) @@ -51,19 +52,19 @@ class JobLogger(val user: String, val logDirName: String) "/tmp/spark-%s".format(user) } - private val jobIDToPrintWriter = new HashMap[Int, PrintWriter] - private val stageIDToJobID = new HashMap[Int, Int] - private val jobIDToStages = new HashMap[Int, ListBuffer[Stage]] + private val jobIdToPrintWriter = new HashMap[Int, PrintWriter] + private val stageIdToJobId = new HashMap[Int, Int] + private val jobIdToStageIds = new HashMap[Int, Seq[Int]] private val DATE_FORMAT = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") - private val eventQueue = new LinkedBlockingQueue[SparkListenerEvents] + private val eventQueue = new LinkedBlockingQueue[SparkListenerEvent] createLogDir() // The following 5 functions are used only in testing. private[scheduler] def getLogDir = logDir - private[scheduler] def getJobIDtoPrintWriter = jobIDToPrintWriter - private[scheduler] def getStageIDToJobID = stageIDToJobID - private[scheduler] def getJobIDToStages = jobIDToStages + private[scheduler] def getJobIdToPrintWriter = jobIdToPrintWriter + private[scheduler] def getStageIdToJobId = stageIdToJobId + private[scheduler] def getJobIdToStageIds = jobIdToStageIds private[scheduler] def getEventQueue = eventQueue /** Create a folder for log files, the folder's name is the creation time of jobLogger */ @@ -80,187 +81,78 @@ class JobLogger(val user: String, val logDirName: String) /** * Create a log file for one job - * @param jobID ID of the job + * @param jobId ID of the job * @throws FileNotFoundException Fail to create log file */ - protected def createLogWriter(jobID: Int) { + protected def createLogWriter(jobId: Int) { try { - val fileWriter = new PrintWriter(logDir + "/" + logDirName + "/" + jobID) - jobIDToPrintWriter += (jobID -> fileWriter) + val fileWriter = new PrintWriter(logDir + "/" + logDirName + "/" + jobId) + jobIdToPrintWriter += (jobId -> fileWriter) } catch { case e: FileNotFoundException => e.printStackTrace() } } /** - * Close log file, and clean the stage relationship in stageIDToJobID - * @param jobID ID of the job + * Close log file, and clean the stage relationship in stageIdToJobId + * @param jobId ID of the job */ - protected def closeLogWriter(jobID: Int) { - jobIDToPrintWriter.get(jobID).foreach { fileWriter => + protected def closeLogWriter(jobId: Int) { + jobIdToPrintWriter.get(jobId).foreach { fileWriter => fileWriter.close() - jobIDToStages.get(jobID).foreach(_.foreach{ stage => - stageIDToJobID -= stage.id + jobIdToStageIds.get(jobId).foreach(_.foreach { stageId => + stageIdToJobId -= stageId }) - jobIDToPrintWriter -= jobID - jobIDToStages -= jobID + jobIdToPrintWriter -= jobId + jobIdToStageIds -= jobId } } + /** + * Build up the maps that represent stage-job relationships + * @param jobId ID of the job + * @param stageIds IDs of the associated stages + */ + protected def buildJobStageDependencies(jobId: Int, stageIds: Seq[Int]) = { + jobIdToStageIds(jobId) = stageIds + stageIds.foreach { stageId => stageIdToJobId(stageId) = jobId } + } + /** * Write info into log file - * @param jobID ID of the job + * @param jobId ID of the job * @param info Info to be recorded * @param withTime Controls whether to record time stamp before the info, default is true */ - protected def jobLogInfo(jobID: Int, info: String, withTime: Boolean = true) { + protected def jobLogInfo(jobId: Int, info: String, withTime: Boolean = true) { var writeInfo = info if (withTime) { val date = new Date(System.currentTimeMillis()) writeInfo = DATE_FORMAT.format(date) + ": " + info } - jobIDToPrintWriter.get(jobID).foreach(_.println(writeInfo)) + jobIdToPrintWriter.get(jobId).foreach(_.println(writeInfo)) } /** * Write info into log file - * @param stageID ID of the stage + * @param stageId ID of the stage * @param info Info to be recorded * @param withTime Controls whether to record time stamp before the info, default is true */ - protected def stageLogInfo(stageID: Int, info: String, withTime: Boolean = true) { - stageIDToJobID.get(stageID).foreach(jobID => jobLogInfo(jobID, info, withTime)) - } - - /** - * Build stage dependency for a job - * @param jobID ID of the job - * @param stage Root stage of the job - */ - protected def buildJobDep(jobID: Int, stage: Stage) { - if (stage.jobId == jobID) { - jobIDToStages.get(jobID) match { - case Some(stageList) => stageList += stage - case None => val stageList = new ListBuffer[Stage] - stageList += stage - jobIDToStages += (jobID -> stageList) - } - stageIDToJobID += (stage.id -> jobID) - stage.parents.foreach(buildJobDep(jobID, _)) - } - } - - /** - * Record stage dependency and RDD dependency for a stage - * @param jobID Job ID of the stage - */ - protected def recordStageDep(jobID: Int) { - def getRddsInStage(rdd: RDD[_]): ListBuffer[RDD[_]] = { - var rddList = new ListBuffer[RDD[_]] - rddList += rdd - rdd.dependencies.foreach { - case shufDep: ShuffleDependency[_, _] => - case dep: Dependency[_] => rddList ++= getRddsInStage(dep.rdd) - } - rddList - } - jobIDToStages.get(jobID).foreach {_.foreach { stage => - var depRddDesc: String = "" - getRddsInStage(stage.rdd).foreach { rdd => - depRddDesc += rdd.id + "," - } - var depStageDesc: String = "" - stage.parents.foreach { stage => - depStageDesc += "(" + stage.id + "," + stage.shuffleDep.get.shuffleId + ")" - } - jobLogInfo(jobID, "STAGE_ID=" + stage.id + " RDD_DEP=(" + - depRddDesc.substring(0, depRddDesc.length - 1) + ")" + - " STAGE_DEP=" + depStageDesc, false) - } - } - } - - /** - * Generate indents and convert to String - * @param indent Number of indents - * @return string of indents - */ - protected def indentString(indent: Int): String = { - val sb = new StringBuilder() - for (i <- 1 to indent) { - sb.append(" ") - } - sb.toString() - } - - /** - * Get RDD's name - * @param rdd Input RDD - * @return String of RDD's name - */ - protected def getRddName(rdd: RDD[_]): String = { - var rddName = rdd.getClass.getSimpleName - if (rdd.name != null) { - rddName = rdd.name - } - rddName - } - - /** - * Record RDD dependency graph in a stage - * @param jobID Job ID of the stage - * @param rdd Root RDD of the stage - * @param indent Indent number before info - */ - protected def recordRddInStageGraph(jobID: Int, rdd: RDD[_], indent: Int) { - val cacheStr = if (rdd.getStorageLevel != StorageLevel.NONE) "CACHED" else "NONE" - val rddInfo = - s"RDD_ID=$rdd.id ${getRddName(rdd)} $cacheStr " + - s"${rdd.getCreationSite} ${rdd.creationSiteInfo.firstUserClass}" - jobLogInfo(jobID, indentString(indent) + rddInfo, false) - rdd.dependencies.foreach { - case shufDep: ShuffleDependency[_, _] => - val depInfo = "SHUFFLE_ID=" + shufDep.shuffleId - jobLogInfo(jobID, indentString(indent + 1) + depInfo, false) - case dep: Dependency[_] => recordRddInStageGraph(jobID, dep.rdd, indent + 1) - } - } - - /** - * Record stage dependency graph of a job - * @param jobID Job ID of the stage - * @param stage Root stage of the job - * @param indent Indent number before info, default is 0 - */ - protected def recordStageDepGraph(jobID: Int, stage: Stage, idSet: HashSet[Int], indent: Int = 0) - { - val stageInfo = if (stage.isShuffleMap) { - "STAGE_ID=" + stage.id + " MAP_STAGE SHUFFLE_ID=" + stage.shuffleDep.get.shuffleId - } else { - "STAGE_ID=" + stage.id + " RESULT_STAGE" - } - if (stage.jobId == jobID) { - jobLogInfo(jobID, indentString(indent) + stageInfo, false) - if (!idSet.contains(stage.id)) { - idSet += stage.id - recordRddInStageGraph(jobID, stage.rdd, indent) - stage.parents.foreach(recordStageDepGraph(jobID, _, idSet, indent + 2)) - } - } else { - jobLogInfo(jobID, indentString(indent) + stageInfo + " JOB_ID=" + stage.jobId, false) - } + protected def stageLogInfo(stageId: Int, info: String, withTime: Boolean = true) { + stageIdToJobId.get(stageId).foreach(jobId => jobLogInfo(jobId, info, withTime)) } /** * Record task metrics into job log files, including execution info and shuffle metrics - * @param stageID Stage ID of the task + * @param stageId Stage ID of the task * @param status Status info of the task * @param taskInfo Task description info * @param taskMetrics Task running metrics */ - protected def recordTaskMetrics(stageID: Int, status: String, + protected def recordTaskMetrics(stageId: Int, status: String, taskInfo: TaskInfo, taskMetrics: TaskMetrics) { - val info = " TID=" + taskInfo.taskId + " STAGE_ID=" + stageID + + val info = " TID=" + taskInfo.taskId + " STAGE_ID=" + stageId + " START_TIME=" + taskInfo.launchTime + " FINISH_TIME=" + taskInfo.finishTime + " EXECUTOR_ID=" + taskInfo.executorId + " HOST=" + taskMetrics.hostname val executorRunTime = " EXECUTOR_RUN_TIME=" + taskMetrics.executorRunTime @@ -278,7 +170,7 @@ class JobLogger(val user: String, val logDirName: String) case Some(metrics) => " SHUFFLE_BYTES_WRITTEN=" + metrics.shuffleBytesWritten case None => "" } - stageLogInfo(stageID, status + info + executorRunTime + readMetrics + writeMetrics) + stageLogInfo(stageId, status + info + executorRunTime + readMetrics + writeMetrics) } /** @@ -286,8 +178,9 @@ class JobLogger(val user: String, val logDirName: String) * @param stageSubmitted Stage submitted event */ override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) { - stageLogInfo(stageSubmitted.stage.stageId,"STAGE_ID=%d STATUS=SUBMITTED TASK_SIZE=%d".format( - stageSubmitted.stage.stageId, stageSubmitted.stage.numTasks)) + val stageInfo = stageSubmitted.stageInfo + stageLogInfo(stageInfo.stageId, "STAGE_ID=%d STATUS=SUBMITTED TASK_SIZE=%d".format( + stageInfo.stageId, stageInfo.numTasks)) } /** @@ -295,36 +188,30 @@ class JobLogger(val user: String, val logDirName: String) * @param stageCompleted Stage completed event */ override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { - stageLogInfo(stageCompleted.stage.stageId, "STAGE_ID=%d STATUS=COMPLETED".format( - stageCompleted.stage.stageId)) + val stageId = stageCompleted.stageInfo.stageId + stageLogInfo(stageId, "STAGE_ID=%d STATUS=COMPLETED".format(stageId)) } - override def onTaskStart(taskStart: SparkListenerTaskStart) { } - /** * When task ends, record task completion status and metrics * @param taskEnd Task end event */ override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { - val task = taskEnd.task val taskInfo = taskEnd.taskInfo - var taskStatus = "" - task match { - case resultTask: ResultTask[_, _] => taskStatus = "TASK_TYPE=RESULT_TASK" - case shuffleMapTask: ShuffleMapTask => taskStatus = "TASK_TYPE=SHUFFLE_MAP_TASK" - } + var taskStatus = "TASK_TYPE=%s".format(taskEnd.taskType) + val taskMetrics = if (taskEnd.taskMetrics != null) taskEnd.taskMetrics else TaskMetrics.empty() taskEnd.reason match { case Success => taskStatus += " STATUS=SUCCESS" - recordTaskMetrics(task.stageId, taskStatus, taskInfo, taskEnd.taskMetrics) + recordTaskMetrics(taskEnd.stageId, taskStatus, taskInfo, taskMetrics) case Resubmitted => taskStatus += " STATUS=RESUBMITTED TID=" + taskInfo.taskId + - " STAGE_ID=" + task.stageId - stageLogInfo(task.stageId, taskStatus) + " STAGE_ID=" + taskEnd.stageId + stageLogInfo(taskEnd.stageId, taskStatus) case FetchFailed(bmAddress, shuffleId, mapId, reduceId) => taskStatus += " STATUS=FETCHFAILED TID=" + taskInfo.taskId + " STAGE_ID=" + - task.stageId + " SHUFFLE_ID=" + shuffleId + " MAP_ID=" + + taskEnd.stageId + " SHUFFLE_ID=" + shuffleId + " MAP_ID=" + mapId + " REDUCE_ID=" + reduceId - stageLogInfo(task.stageId, taskStatus) + stageLogInfo(taskEnd.stageId, taskStatus) case _ => } } @@ -334,8 +221,8 @@ class JobLogger(val user: String, val logDirName: String) * @param jobEnd Job end event */ override def onJobEnd(jobEnd: SparkListenerJobEnd) { - val job = jobEnd.job - var info = "JOB_ID=" + job.jobId + val jobId = jobEnd.jobId + var info = "JOB_ID=" + jobId jobEnd.jobResult match { case JobSucceeded => info += " STATUS=SUCCESS" case JobFailed(exception, _) => @@ -343,19 +230,19 @@ class JobLogger(val user: String, val logDirName: String) exception.getMessage.split("\\s+").foreach(info += _ + "_") case _ => } - jobLogInfo(job.jobId, info.substring(0, info.length - 1).toUpperCase) - closeLogWriter(job.jobId) + jobLogInfo(jobId, info.substring(0, info.length - 1).toUpperCase) + closeLogWriter(jobId) } /** * Record job properties into job log file - * @param jobID ID of the job + * @param jobId ID of the job * @param properties Properties of the job */ - protected def recordJobProperties(jobID: Int, properties: Properties) { - if(properties != null) { + protected def recordJobProperties(jobId: Int, properties: Properties) { + if (properties != null) { val description = properties.getProperty(SparkContext.SPARK_JOB_DESCRIPTION, "") - jobLogInfo(jobID, description, false) + jobLogInfo(jobId, description, false) } } @@ -364,14 +251,11 @@ class JobLogger(val user: String, val logDirName: String) * @param jobStart Job start event */ override def onJobStart(jobStart: SparkListenerJobStart) { - val job = jobStart.job + val jobId = jobStart.jobId val properties = jobStart.properties - createLogWriter(job.jobId) - recordJobProperties(job.jobId, properties) - buildJobDep(job.jobId, job.finalStage) - recordStageDep(job.jobId) - recordStageDepGraph(job.jobId, job.finalStage, new HashSet[Int]) - jobLogInfo(job.jobId, "JOB_ID=" + job.jobId + " STATUS=STARTED") + createLogWriter(jobId) + recordJobProperties(jobId, properties) + buildJobStageDependencies(jobId, jobStart.stageIds) + jobLogInfo(jobId, "JOB_ID=" + jobId + " STATUS=STARTED") } } - diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala index d94f6ad924260..3cf4e3077e4a4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala @@ -23,5 +23,6 @@ package org.apache.spark.scheduler private[spark] sealed trait JobResult private[spark] case object JobSucceeded extends JobResult -private[spark] case class JobFailed(exception: Exception, failedStage: Option[Stage]) - extends JobResult + +// A failed stage ID of -1 means there is not a particular stage that caused the failure +private[spark] case class JobFailed(exception: Exception, failedStageId: Int) extends JobResult diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala index b026f860a8cd8..8007b5418741e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala @@ -64,7 +64,7 @@ private[spark] class JobWaiter[T]( override def jobFailed(exception: Exception): Unit = synchronized { _jobFinished = true - jobResult = JobFailed(exception, None) + jobResult = JobFailed(exception, -1) this.notifyAll() } diff --git a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala new file mode 100644 index 0000000000000..353a48661b0f7 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import java.util.concurrent.LinkedBlockingQueue + +import org.apache.spark.Logging + +/** + * Asynchronously passes SparkListenerEvents to registered SparkListeners. + * + * Until start() is called, all posted events are only buffered. Only after this listener bus + * has started will events be actually propagated to all attached listeners. This listener bus + * is stopped when it receives a SparkListenerShutdown event, which is posted using stop(). + */ +private[spark] class LiveListenerBus extends SparkListenerBus with Logging { + + /* Cap the capacity of the SparkListenerEvent queue so we get an explicit error (rather than + * an OOM exception) if it's perpetually being added to more quickly than it's being drained. */ + private val EVENT_QUEUE_CAPACITY = 10000 + private val eventQueue = new LinkedBlockingQueue[SparkListenerEvent](EVENT_QUEUE_CAPACITY) + private var queueFullErrorMessageLogged = false + private var started = false + + /** + * Start sending events to attached listeners. + * + * This first sends out all buffered events posted before this listener bus has started, then + * listens for any additional events asynchronously while the listener bus is still running. + * This should only be called once. + */ + def start() { + if (started) { + throw new IllegalStateException("Listener bus already started!") + } + started = true + new Thread("SparkListenerBus") { + setDaemon(true) + override def run() { + while (true) { + val event = eventQueue.take + if (event == SparkListenerShutdown) { + // Get out of the while loop and shutdown the daemon thread + return + } + postToAll(event) + } + } + }.start() + } + + def post(event: SparkListenerEvent) { + val eventAdded = eventQueue.offer(event) + if (!eventAdded && !queueFullErrorMessageLogged) { + logError("Dropping SparkListenerEvent because no remaining room in event queue. " + + "This likely means one of the SparkListeners is too slow and cannot keep up with the " + + "rate at which tasks are being started by the scheduler.") + queueFullErrorMessageLogged = true + } + } + + /** + * Waits until there are no more events in the queue, or until the specified time has elapsed. + * Used for testing only. Returns true if the queue has emptied and false is the specified time + * elapsed before the queue emptied. + */ + def waitUntilEmpty(timeoutMillis: Int): Boolean = { + val finishTime = System.currentTimeMillis + timeoutMillis + while (!eventQueue.isEmpty) { + if (System.currentTimeMillis > finishTime) { + return false + } + /* Sleep rather than using wait/notify, because this is used only for testing and wait/notify + * add overhead in the general case. */ + Thread.sleep(10) + } + true + } + + def stop() { + if (!started) { + throw new IllegalStateException("Attempted to stop a listener bus that has not yet started!") + } + post(SparkListenerShutdown) + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala index 4bc13c23d980b..187672c4e19e7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala @@ -62,7 +62,7 @@ private[spark] class Pool( override def addSchedulable(schedulable: Schedulable) { schedulableQueue += schedulable schedulableNameToSchedulable(schedulable.name) = schedulable - schedulable.parent= this + schedulable.parent = this } override def removeSchedulable(schedulable: Schedulable) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala new file mode 100644 index 0000000000000..db76178b65501 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import java.io.InputStream +import java.net.URI + +import scala.io.Source + +import it.unimi.dsi.fastutil.io.FastBufferedInputStream +import org.apache.hadoop.fs.{Path, FileSystem} +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.io.CompressionCodec +import org.apache.spark.util.{JsonProtocol, Utils} + +/** + * An EventBus that replays logged events from persisted storage + */ +private[spark] class ReplayListenerBus(conf: SparkConf) extends SparkListenerBus with Logging { + private val compressed = conf.getBoolean("spark.eventLog.compress", false) + + // Only used if compression is enabled + private lazy val compressionCodec = CompressionCodec.createCodec(conf) + + /** + * Return a list of paths representing log files in the given directory. + */ + private def getLogFilePaths(logDir: String, fileSystem: FileSystem): Array[Path] = { + val path = new Path(logDir) + if (!fileSystem.exists(path) || !fileSystem.getFileStatus(path).isDir) { + logWarning("Log path provided is not a valid directory: %s".format(logDir)) + return Array[Path]() + } + val logStatus = fileSystem.listStatus(path) + if (logStatus == null || !logStatus.exists(!_.isDir)) { + logWarning("Log path provided contains no log files: %s".format(logDir)) + return Array[Path]() + } + logStatus.filter(!_.isDir).map(_.getPath).sortBy(_.getName) + } + + /** + * Replay each event in the order maintained in the given logs. + */ + def replay(logDir: String): Boolean = { + val fileSystem = Utils.getHadoopFileSystem(new URI(logDir)) + val logPaths = getLogFilePaths(logDir, fileSystem) + if (logPaths.length == 0) { + return false + } + + logPaths.foreach { path => + // Keep track of input streams at all levels to close them later + // This is necessary because an exception can occur in between stream initializations + var fileStream: Option[InputStream] = None + var bufferedStream: Option[InputStream] = None + var compressStream: Option[InputStream] = None + var currentLine = "" + try { + currentLine = "" + fileStream = Some(fileSystem.open(path)) + bufferedStream = Some(new FastBufferedInputStream(fileStream.get)) + compressStream = + if (compressed) { + Some(compressionCodec.compressedInputStream(bufferedStream.get)) + } else bufferedStream + + // Parse each line as an event and post it to all attached listeners + val lines = Source.fromInputStream(compressStream.get).getLines() + lines.foreach { line => + currentLine = line + postToAll(JsonProtocol.sparkEventFromJson(parse(line))) + } + } catch { + case e: Exception => + logError("Exception in parsing Spark event log %s".format(path), e) + logError("Malformed line: %s\n".format(currentLine)) + } finally { + fileStream.foreach(_.close()) + bufferedStream.foreach(_.close()) + compressStream.foreach(_.close()) + } + } + fileSystem.close() + true + } +} 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 9590c03f10632..d4eb0ac88d8e8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -19,33 +19,52 @@ package org.apache.spark.scheduler import java.util.Properties +import scala.collection.Map +import scala.collection.mutable + import org.apache.spark.{Logging, TaskEndReason} import org.apache.spark.executor.TaskMetrics +import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.{Distribution, Utils} -sealed trait SparkListenerEvents +sealed trait SparkListenerEvent + +case class SparkListenerStageSubmitted(stageInfo: StageInfo, properties: Properties = null) + extends SparkListenerEvent + +case class SparkListenerStageCompleted(stageInfo: StageInfo) extends SparkListenerEvent + +case class SparkListenerTaskStart(stageId: Int, taskInfo: TaskInfo) extends SparkListenerEvent -case class SparkListenerStageSubmitted(stage: StageInfo, properties: Properties) - extends SparkListenerEvents +case class SparkListenerTaskGettingResult(taskInfo: TaskInfo) extends SparkListenerEvent -case class SparkListenerStageCompleted(stage: StageInfo) extends SparkListenerEvents +case class SparkListenerTaskEnd( + stageId: Int, + taskType: String, + reason: TaskEndReason, + taskInfo: TaskInfo, + taskMetrics: TaskMetrics) + extends SparkListenerEvent -case class SparkListenerTaskStart(task: Task[_], taskInfo: TaskInfo) extends SparkListenerEvents +case class SparkListenerJobStart(jobId: Int, stageIds: Seq[Int], properties: Properties = null) + extends SparkListenerEvent -case class SparkListenerTaskGettingResult( - task: Task[_], taskInfo: TaskInfo) extends SparkListenerEvents +case class SparkListenerJobEnd(jobId: Int, jobResult: JobResult) extends SparkListenerEvent -case class SparkListenerTaskEnd(task: Task[_], reason: TaskEndReason, taskInfo: TaskInfo, - taskMetrics: TaskMetrics) extends SparkListenerEvents +case class SparkListenerEnvironmentUpdate(environmentDetails: Map[String, Seq[(String, String)]]) + extends SparkListenerEvent -case class SparkListenerJobStart(job: ActiveJob, stageIds: Array[Int], - properties: Properties = null) extends SparkListenerEvents +case class SparkListenerBlockManagerAdded(blockManagerId: BlockManagerId, maxMem: Long) + extends SparkListenerEvent -case class SparkListenerJobEnd(job: ActiveJob, jobResult: JobResult) - extends SparkListenerEvents +case class SparkListenerBlockManagerRemoved(blockManagerId: BlockManagerId) + extends SparkListenerEvent + +case class SparkListenerUnpersistRDD(rddId: Int) extends SparkListenerEvent /** An event used in the listener to shutdown the listener daemon thread. */ -private[scheduler] case object SparkListenerShutdown extends SparkListenerEvents +private[spark] case object SparkListenerShutdown extends SparkListenerEvent + /** * Interface for listening to events from the Spark scheduler. @@ -87,97 +106,134 @@ trait SparkListener { */ def onJobEnd(jobEnd: SparkListenerJobEnd) { } + /** + * Called when environment properties have been updated + */ + def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { } + + /** + * Called when a new block manager has joined + */ + def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded) { } + + /** + * Called when an existing block manager has been removed + */ + def onBlockManagerRemoved(blockManagerRemoved: SparkListenerBlockManagerRemoved) { } + + /** + * Called when an RDD is manually unpersisted by the application + */ + def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD) { } } /** * Simple SparkListener that logs a few summary statistics when each stage completes */ class StatsReportListener extends SparkListener with Logging { + + import org.apache.spark.scheduler.StatsReportListener._ + + private val taskInfoMetrics = mutable.Buffer[(TaskInfo, TaskMetrics)]() + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + val info = taskEnd.taskInfo + val metrics = taskEnd.taskMetrics + if (info != null && metrics != null) { + taskInfoMetrics += ((info, metrics)) + } + } + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { - import org.apache.spark.scheduler.StatsReportListener._ implicit val sc = stageCompleted - this.logInfo("Finished stage: " + stageCompleted.stage) - showMillisDistribution("task runtime:", (info, _) => Some(info.duration)) + this.logInfo("Finished stage: " + stageCompleted.stageInfo) + showMillisDistribution("task runtime:", (info, _) => Some(info.duration), taskInfoMetrics) - //shuffle write + // Shuffle write showBytesDistribution("shuffle bytes written:", - (_,metric) => metric.shuffleWriteMetrics.map(_.shuffleBytesWritten)) + (_, metric) => metric.shuffleWriteMetrics.map(_.shuffleBytesWritten), taskInfoMetrics) - //fetch & io + // Fetch & I/O showMillisDistribution("fetch wait time:", - (_, metric) => metric.shuffleReadMetrics.map(_.fetchWaitTime)) + (_, metric) => metric.shuffleReadMetrics.map(_.fetchWaitTime), taskInfoMetrics) showBytesDistribution("remote bytes read:", - (_, metric) => metric.shuffleReadMetrics.map(_.remoteBytesRead)) - showBytesDistribution("task result size:", (_, metric) => Some(metric.resultSize)) - - //runtime breakdown + (_, metric) => metric.shuffleReadMetrics.map(_.remoteBytesRead), taskInfoMetrics) + showBytesDistribution("task result size:", + (_, metric) => Some(metric.resultSize), taskInfoMetrics) - val runtimePcts = stageCompleted.stage.taskInfos.map{ - case (info, metrics) => RuntimePercentage(info.duration, metrics) + // Runtime breakdown + val runtimePcts = taskInfoMetrics.map { case (info, metrics) => + RuntimePercentage(info.duration, metrics) } showDistribution("executor (non-fetch) time pct: ", - Distribution(runtimePcts.map{_.executorPct * 100}), "%2.0f %%") + Distribution(runtimePcts.map(_.executorPct * 100)), "%2.0f %%") showDistribution("fetch wait time pct: ", - Distribution(runtimePcts.flatMap{_.fetchPct.map{_ * 100}}), "%2.0f %%") - showDistribution("other time pct: ", Distribution(runtimePcts.map{_.other * 100}), "%2.0f %%") + Distribution(runtimePcts.flatMap(_.fetchPct.map(_ * 100))), "%2.0f %%") + showDistribution("other time pct: ", Distribution(runtimePcts.map(_.other * 100)), "%2.0f %%") + taskInfoMetrics.clear() } } private[spark] object StatsReportListener extends Logging { - //for profiling, the extremes are more interesting + // For profiling, the extremes are more interesting val percentiles = Array[Int](0,5,10,25,50,75,90,95,100) - val probabilities = percentiles.map{_ / 100.0} + val probabilities = percentiles.map(_ / 100.0) val percentilesHeader = "\t" + percentiles.mkString("%\t") + "%" - def extractDoubleDistribution(stage: SparkListenerStageCompleted, - getMetric: (TaskInfo,TaskMetrics) => Option[Double]) - : Option[Distribution] = { - Distribution(stage.stage.taskInfos.flatMap { - case ((info,metric)) => getMetric(info, metric)}) + def extractDoubleDistribution( + taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)], + getMetric: (TaskInfo, TaskMetrics) => Option[Double]): Option[Distribution] = { + Distribution(taskInfoMetrics.flatMap { case (info, metric) => getMetric(info, metric) }) } - //is there some way to setup the types that I can get rid of this completely? - def extractLongDistribution(stage: SparkListenerStageCompleted, - getMetric: (TaskInfo,TaskMetrics) => Option[Long]) - : Option[Distribution] = { - extractDoubleDistribution(stage, (info, metric) => getMetric(info,metric).map{_.toDouble}) + // Is there some way to setup the types that I can get rid of this completely? + def extractLongDistribution( + taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)], + getMetric: (TaskInfo, TaskMetrics) => Option[Long]): Option[Distribution] = { + extractDoubleDistribution( + taskInfoMetrics, + (info, metric) => { getMetric(info, metric).map(_.toDouble) }) } def showDistribution(heading: String, d: Distribution, formatNumber: Double => String) { val stats = d.statCounter - val quantiles = d.getQuantiles(probabilities).map{formatNumber} + val quantiles = d.getQuantiles(probabilities).map(formatNumber) logInfo(heading + stats) logInfo(percentilesHeader) logInfo("\t" + quantiles.mkString("\t")) } - def showDistribution(heading: String, dOpt: Option[Distribution], formatNumber: Double => String) - { + def showDistribution( + heading: String, + dOpt: Option[Distribution], + formatNumber: Double => String) { dOpt.foreach { d => showDistribution(heading, d, formatNumber)} } def showDistribution(heading: String, dOpt: Option[Distribution], format:String) { - def f(d:Double) = format.format(d) + def f(d: Double) = format.format(d) showDistribution(heading, dOpt, f _) } def showDistribution( heading: String, format: String, - getMetric: (TaskInfo, TaskMetrics) => Option[Double]) - (implicit stage: SparkListenerStageCompleted) { - showDistribution(heading, extractDoubleDistribution(stage, getMetric), format) + getMetric: (TaskInfo, TaskMetrics) => Option[Double], + taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)]) { + showDistribution(heading, extractDoubleDistribution(taskInfoMetrics, getMetric), format) } - def showBytesDistribution(heading:String, getMetric: (TaskInfo,TaskMetrics) => Option[Long]) - (implicit stage: SparkListenerStageCompleted) { - showBytesDistribution(heading, extractLongDistribution(stage, getMetric)) + def showBytesDistribution( + heading:String, + getMetric: (TaskInfo, TaskMetrics) => Option[Long], + taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)]) { + showBytesDistribution(heading, extractLongDistribution(taskInfoMetrics, getMetric)) } def showBytesDistribution(heading: String, dOpt: Option[Distribution]) { - dOpt.foreach{dist => showBytesDistribution(heading, dist)} + dOpt.foreach { dist => showBytesDistribution(heading, dist) } } def showBytesDistribution(heading: String, dist: Distribution) { @@ -189,9 +245,11 @@ private[spark] object StatsReportListener extends Logging { (d => StatsReportListener.millisToString(d.toLong)): Double => String) } - def showMillisDistribution(heading: String, getMetric: (TaskInfo, TaskMetrics) => Option[Long]) - (implicit stage: SparkListenerStageCompleted) { - showMillisDistribution(heading, extractLongDistribution(stage, getMetric)) + def showMillisDistribution( + heading: String, + getMetric: (TaskInfo, TaskMetrics) => Option[Long], + taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)]) { + showMillisDistribution(heading, extractLongDistribution(taskInfoMetrics, getMetric)) } val seconds = 1000L @@ -199,7 +257,7 @@ private[spark] object StatsReportListener extends Logging { val hours = minutes * 60 /** - * reformat a time interval in milliseconds to a prettier format for output + * Reformat a time interval in milliseconds to a prettier format for output */ def millisToString(ms: Long) = { val (size, units) = @@ -221,8 +279,8 @@ private case class RuntimePercentage(executorPct: Double, fetchPct: Option[Doubl private object RuntimePercentage { def apply(totalTime: Long, metrics: TaskMetrics): RuntimePercentage = { val denom = totalTime.toDouble - val fetchTime = metrics.shuffleReadMetrics.map{_.fetchWaitTime} - val fetch = fetchTime.map{_ / denom} + val fetchTime = metrics.shuffleReadMetrics.map(_.fetchWaitTime) + val fetch = fetchTime.map(_ / denom) val exec = (metrics.executorRunTime - fetchTime.getOrElse(0L)) / denom val other = 1.0 - (exec + fetch.getOrElse(0d)) RuntimePercentage(exec, fetch, other) diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index 17b1328b86788..729e120497571 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -1,100 +1,67 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.scheduler - -import java.util.concurrent.LinkedBlockingQueue - -import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer} - -import org.apache.spark.Logging - -/** Asynchronously passes SparkListenerEvents to registered SparkListeners. */ -private[spark] class SparkListenerBus extends Logging { - private val sparkListeners = new ArrayBuffer[SparkListener] with SynchronizedBuffer[SparkListener] - - /* Cap the capacity of the SparkListenerEvent queue so we get an explicit error (rather than - * an OOM exception) if it's perpetually being added to more quickly than it's being drained. */ - private val EVENT_QUEUE_CAPACITY = 10000 - private val eventQueue = new LinkedBlockingQueue[SparkListenerEvents](EVENT_QUEUE_CAPACITY) - private var queueFullErrorMessageLogged = false - - // Create a new daemon thread to listen for events. This thread is stopped when it receives - // a SparkListenerShutdown event, using the stop method. - new Thread("SparkListenerBus") { - setDaemon(true) - override def run() { - while (true) { - val event = eventQueue.take - event match { - case stageSubmitted: SparkListenerStageSubmitted => - sparkListeners.foreach(_.onStageSubmitted(stageSubmitted)) - case stageCompleted: SparkListenerStageCompleted => - sparkListeners.foreach(_.onStageCompleted(stageCompleted)) - case jobStart: SparkListenerJobStart => - sparkListeners.foreach(_.onJobStart(jobStart)) - case jobEnd: SparkListenerJobEnd => - sparkListeners.foreach(_.onJobEnd(jobEnd)) - case taskStart: SparkListenerTaskStart => - sparkListeners.foreach(_.onTaskStart(taskStart)) - case taskGettingResult: SparkListenerTaskGettingResult => - sparkListeners.foreach(_.onTaskGettingResult(taskGettingResult)) - case taskEnd: SparkListenerTaskEnd => - sparkListeners.foreach(_.onTaskEnd(taskEnd)) - case SparkListenerShutdown => - // Get out of the while loop and shutdown the daemon thread - return - case _ => - } - } - } - }.start() - - def addListener(listener: SparkListener) { - sparkListeners += listener - } - - def post(event: SparkListenerEvents) { - val eventAdded = eventQueue.offer(event) - if (!eventAdded && !queueFullErrorMessageLogged) { - logError("Dropping SparkListenerEvent because no remaining room in event queue. " + - "This likely means one of the SparkListeners is too slow and cannot keep up with the " + - "rate at which tasks are being started by the scheduler.") - queueFullErrorMessageLogged = true - } - } - - /** - * Waits until there are no more events in the queue, or until the specified time has elapsed. - * Used for testing only. Returns true if the queue has emptied and false is the specified time - * elapsed before the queue emptied. - */ - def waitUntilEmpty(timeoutMillis: Int): Boolean = { - val finishTime = System.currentTimeMillis + timeoutMillis - while (!eventQueue.isEmpty) { - if (System.currentTimeMillis > finishTime) { - return false - } - /* Sleep rather than using wait/notify, because this is used only for testing and wait/notify - * add overhead in the general case. */ - Thread.sleep(10) - } - true - } - - def stop(): Unit = post(SparkListenerShutdown) -} +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer + +/** + * A SparkListenerEvent bus that relays events to its listeners + */ +private[spark] trait SparkListenerBus { + + // SparkListeners attached to this event bus + protected val sparkListeners = new ArrayBuffer[SparkListener] + with mutable.SynchronizedBuffer[SparkListener] + + def addListener(listener: SparkListener) { + sparkListeners += listener + } + + /** + * Post an event to all attached listeners. This does nothing if the event is + * SparkListenerShutdown. + */ + protected def postToAll(event: SparkListenerEvent) { + event match { + case stageSubmitted: SparkListenerStageSubmitted => + sparkListeners.foreach(_.onStageSubmitted(stageSubmitted)) + case stageCompleted: SparkListenerStageCompleted => + sparkListeners.foreach(_.onStageCompleted(stageCompleted)) + case jobStart: SparkListenerJobStart => + sparkListeners.foreach(_.onJobStart(jobStart)) + case jobEnd: SparkListenerJobEnd => + sparkListeners.foreach(_.onJobEnd(jobEnd)) + case taskStart: SparkListenerTaskStart => + sparkListeners.foreach(_.onTaskStart(taskStart)) + case taskGettingResult: SparkListenerTaskGettingResult => + sparkListeners.foreach(_.onTaskGettingResult(taskGettingResult)) + case taskEnd: SparkListenerTaskEnd => + sparkListeners.foreach(_.onTaskEnd(taskEnd)) + case environmentUpdate: SparkListenerEnvironmentUpdate => + sparkListeners.foreach(_.onEnvironmentUpdate(environmentUpdate)) + case blockManagerAdded: SparkListenerBlockManagerAdded => + sparkListeners.foreach(_.onBlockManagerAdded(blockManagerAdded)) + case blockManagerRemoved: SparkListenerBlockManagerRemoved => + sparkListeners.foreach(_.onBlockManagerRemoved(blockManagerRemoved)) + case unpersistRDD: SparkListenerUnpersistRDD => + sparkListeners.foreach(_.onUnpersistRDD(unpersistRDD)) + case SparkListenerShutdown => + } + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index 8f320e5c7a74b..8115a7ed7896d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -17,28 +17,25 @@ package org.apache.spark.scheduler -import scala.collection._ - -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.storage.RDDInfo /** * Stores information about a stage to pass from the scheduler to SparkListeners. - * - * taskInfos stores the metrics for all tasks that have completed, including redundant, speculated - * tasks. */ -class StageInfo( - stage: Stage, - val taskInfos: mutable.Buffer[(TaskInfo, TaskMetrics)] = - mutable.Buffer[(TaskInfo, TaskMetrics)]() -) { - val stageId = stage.id +private[spark] +class StageInfo(val stageId: Int, val name: String, val numTasks: Int, val rddInfo: RDDInfo) { /** When this stage was submitted from the DAGScheduler to a TaskScheduler. */ var submissionTime: Option[Long] = None var completionTime: Option[Long] = None - val rddName = stage.rdd.name - val name = stage.name - val numPartitions = stage.numPartitions - val numTasks = stage.numTasks var emittedTaskSizeWarning = false } + +private[spark] +object StageInfo { + def fromStage(stage: Stage): StageInfo = { + val rdd = stage.rdd + val rddName = Option(rdd.name).getOrElse(rdd.id.toString) + val rddInfo = new RDDInfo(rdd.id, rddName, rdd.partitions.size, rdd.getStorageLevel) + new StageInfo(stage.id, stage.name, stage.numTasks, rddInfo) + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala index ea3229b75be36..308edb12edd5c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala @@ -18,7 +18,7 @@ package org.apache.spark.scheduler private[spark] object TaskLocality extends Enumeration { - // process local is expected to be used ONLY within tasksetmanager for now. + // Process local is expected to be used ONLY within TaskSetManager for now. val PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY = Value type TaskLocality = Value 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 abff252597e16..30bceb47b9e7d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -204,7 +204,7 @@ private[spark] class TaskSchedulerImpl( executorIdToHost(o.executorId) = o.host if (!executorsByHost.contains(o.host)) { executorsByHost(o.host) = new HashSet[String]() - executorGained(o.executorId, o.host) + executorAdded(o.executorId, o.host) } } @@ -400,8 +400,8 @@ private[spark] class TaskSchedulerImpl( rootPool.executorLost(executorId, host) } - def executorGained(execId: String, host: String) { - dagScheduler.executorGained(execId, host) + def executorAdded(execId: String, host: String) { + dagScheduler.executorAdded(execId, host) } def getExecutorsAliveOnHost(host: String): Option[Set[String]] = synchronized { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index ee4b65e312abc..25b7472a99cdb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -18,7 +18,7 @@ package org.apache.spark.scheduler.cluster import org.apache.spark.{Logging, SparkContext} -import org.apache.spark.deploy.{Command, ApplicationDescription} +import org.apache.spark.deploy.{ApplicationDescription, Command} import org.apache.spark.deploy.client.{AppClient, AppClientListener} import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason, SlaveLost, TaskSchedulerImpl} import org.apache.spark.util.Utils @@ -26,8 +26,7 @@ import org.apache.spark.util.Utils private[spark] class SparkDeploySchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext, - masters: Array[String], - appName: String) + masters: Array[String]) extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) with AppClientListener with Logging { @@ -49,8 +48,8 @@ private[spark] class SparkDeploySchedulerBackend( val command = Command( "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs) val sparkHome = sc.getSparkHome() - val appDesc = new ApplicationDescription(appName, maxCores, sc.executorMemory, command, - sparkHome, "http://" + sc.ui.appUIAddress) + val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, + sparkHome, sc.ui.appUIAddress, sc.eventLoggingInfo) client = new AppClient(sc.env.actorSystem, masters, appDesc, this, conf) client.start() diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 28b019d9fd495..06b041e1fd9a9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -45,8 +45,7 @@ import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend private[spark] class CoarseMesosSchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext, - master: String, - appName: String) + master: String) extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) with MScheduler with Logging { @@ -94,7 +93,7 @@ private[spark] class CoarseMesosSchedulerBackend( setDaemon(true) override def run() { val scheduler = CoarseMesosSchedulerBackend.this - val fwInfo = FrameworkInfo.newBuilder().setUser("").setName(appName).build() + val fwInfo = FrameworkInfo.newBuilder().setUser("").setName(sc.appName).build() driver = new MesosSchedulerDriver(scheduler, fwInfo, master) try { { val ret = driver.run() diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index bcf0ce19a54cd..4092dd04b112b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -41,8 +41,7 @@ import org.apache.spark.util.Utils private[spark] class MesosSchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext, - master: String, - appName: String) + master: String) extends SchedulerBackend with MScheduler with Logging { @@ -71,7 +70,7 @@ private[spark] class MesosSchedulerBackend( setDaemon(true) override def run() { val scheduler = MesosSchedulerBackend.this - val fwInfo = FrameworkInfo.newBuilder().setUser("").setName(appName).build() + val fwInfo = FrameworkInfo.newBuilder().setUser("").setName(sc.appName).build() driver = new MesosSchedulerDriver(scheduler, fwInfo, master) try { val ret = driver.run() diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 1bf3f4db32ea7..71584b6eb102a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -29,7 +29,7 @@ import akka.actor.{ActorSystem, Cancellable, Props} import it.unimi.dsi.fastutil.io.{FastBufferedOutputStream, FastByteArrayOutputStream} import sun.nio.ch.DirectBuffer -import org.apache.spark.{Logging, SparkConf, SparkEnv, SparkException, SecurityManager} +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkEnv, SparkException} import org.apache.spark.io.CompressionCodec import org.apache.spark.network._ import org.apache.spark.serializer.Serializer @@ -92,7 +92,7 @@ private[spark] class BlockManager( val slaveActor = actorSystem.actorOf(Props(new BlockManagerSlaveActor(this)), name = "BlockManagerActor" + BlockManager.ID_GENERATOR.next) - // Pending reregistration action being executed asynchronously or null if none + // Pending re-registration action being executed asynchronously or null if none // is pending. Accesses should synchronize on asyncReregisterLock. var asyncReregisterTask: Future[Unit] = null val asyncReregisterLock = new Object @@ -122,10 +122,15 @@ private[spark] class BlockManager( /** * Construct a BlockManager with a memory limit set based on system properties. */ - def this(execId: String, actorSystem: ActorSystem, master: BlockManagerMaster, - serializer: Serializer, conf: SparkConf, securityManager: SecurityManager) = { - this(execId, actorSystem, master, serializer, BlockManager.getMaxMemory(conf), conf, - securityManager) + def this( + execId: String, + actorSystem: ActorSystem, + master: BlockManagerMaster, + serializer: Serializer, + conf: SparkConf, + securityManager: SecurityManager) = { + this(execId, actorSystem, master, serializer, BlockManager.getMaxMemory(conf), + conf, securityManager) } /** @@ -148,14 +153,15 @@ private[spark] class BlockManager( * an executor crash. * * This function deliberately fails silently if the master returns false (indicating that - * the slave needs to reregister). The error condition will be detected again by the next - * heart beat attempt or new block registration and another try to reregister all blocks + * the slave needs to re-register). The error condition will be detected again by the next + * heart beat attempt or new block registration and another try to re-register all blocks * will be made then. */ private def reportAllBlocks() { logInfo("Reporting " + blockInfo.size + " blocks to the master.") for ((blockId, info) <- blockInfo) { - if (!tryToReportBlockStatus(blockId, info)) { + val status = getCurrentBlockStatus(blockId, info) + if (!tryToReportBlockStatus(blockId, info, status)) { logError("Failed to report " + blockId + " to master; giving up.") return } @@ -163,20 +169,20 @@ private[spark] class BlockManager( } /** - * Reregister with the master and report all blocks to it. This will be called by the heart beat + * Re-register with the master and report all blocks to it. This will be called by the heart beat * thread if our heartbeat to the block manager indicates that we were not registered. * * Note that this method must be called without any BlockInfo locks held. */ def reregister() { - // TODO: We might need to rate limit reregistering. - logInfo("BlockManager reregistering with master") + // TODO: We might need to rate limit re-registering. + logInfo("BlockManager re-registering with master") master.registerBlockManager(blockManagerId, maxMemory, slaveActor) reportAllBlocks() } /** - * Reregister with the master sometime soon. + * Re-register with the master sometime soon. */ def asyncReregister() { asyncReregisterLock.synchronized { @@ -192,7 +198,7 @@ private[spark] class BlockManager( } /** - * For testing. Wait for any pending asynchronous reregistration; otherwise, do nothing. + * For testing. Wait for any pending asynchronous re-registration; otherwise, do nothing. */ def waitForAsyncReregister() { val task = asyncReregisterTask @@ -211,15 +217,19 @@ private[spark] class BlockManager( * message reflecting the current status, *not* the desired storage level in its block info. * For example, a block with MEMORY_AND_DISK set might have fallen out to be only on disk. * - * droppedMemorySize exists to account for when block is dropped from memory to disk (so it - * is still valid). This ensures that update in master will compensate for the increase in + * droppedMemorySize exists to account for when the block is dropped from memory to disk (so + * it is still valid). This ensures that update in master will compensate for the increase in * memory on slave. */ - def reportBlockStatus(blockId: BlockId, info: BlockInfo, droppedMemorySize: Long = 0L) { - val needReregister = !tryToReportBlockStatus(blockId, info, droppedMemorySize) + def reportBlockStatus( + blockId: BlockId, + info: BlockInfo, + status: BlockStatus, + droppedMemorySize: Long = 0L) { + val needReregister = !tryToReportBlockStatus(blockId, info, status, droppedMemorySize) if (needReregister) { - logInfo("Got told to reregister updating block " + blockId) - // Reregistering will report our new block for free. + logInfo("Got told to re-register updating block " + blockId) + // Re-registering will report our new block for free. asyncReregister() } logDebug("Told master about block " + blockId) @@ -230,27 +240,41 @@ private[spark] class BlockManager( * which will be true if the block was successfully recorded and false if * the slave needs to re-register. */ - private def tryToReportBlockStatus(blockId: BlockId, info: BlockInfo, + private def tryToReportBlockStatus( + blockId: BlockId, + info: BlockInfo, + status: BlockStatus, droppedMemorySize: Long = 0L): Boolean = { - val (curLevel, inMemSize, onDiskSize, tellMaster) = info.synchronized { + if (info.tellMaster) { + val storageLevel = status.storageLevel + val inMemSize = Math.max(status.memSize, droppedMemorySize) + val onDiskSize = status.diskSize + master.updateBlockInfo(blockManagerId, blockId, storageLevel, inMemSize, onDiskSize) + } else true + } + + /** + * Return the updated storage status of the block with the given ID. More specifically, if + * the block is dropped from memory and possibly added to disk, return the new storage level + * and the updated in-memory and on-disk sizes. + */ + private def getCurrentBlockStatus(blockId: BlockId, info: BlockInfo): BlockStatus = { + val (newLevel, inMemSize, onDiskSize) = info.synchronized { info.level match { case null => - (StorageLevel.NONE, 0L, 0L, false) + (StorageLevel.NONE, 0L, 0L) case level => val inMem = level.useMemory && memoryStore.contains(blockId) val onDisk = level.useDisk && diskStore.contains(blockId) - val storageLevel = StorageLevel(onDisk, inMem, level.deserialized, level.replication) - val memSize = if (inMem) memoryStore.getSize(blockId) else droppedMemorySize + val deserialized = if (inMem) level.deserialized else false + val replication = if (inMem || onDisk) level.replication else 1 + val storageLevel = StorageLevel(onDisk, inMem, deserialized, replication) + val memSize = if (inMem) memoryStore.getSize(blockId) else 0L val diskSize = if (onDisk) diskStore.getSize(blockId) else 0L - (storageLevel, memSize, diskSize, info.tellMaster) + (storageLevel, memSize, diskSize) } } - - if (tellMaster) { - master.updateBlockInfo(blockManagerId, blockId, curLevel, inMemSize, onDiskSize) - } else { - true - } + BlockStatus(newLevel, inMemSize, onDiskSize) } /** @@ -398,10 +422,10 @@ private[spark] class BlockManager( /** * Get block from remote block managers as serialized bytes. */ - def getRemoteBytes(blockId: BlockId): Option[ByteBuffer] = { + def getRemoteBytes(blockId: BlockId): Option[ByteBuffer] = { logDebug("Getting remote block " + blockId + " as bytes") doGetRemote(blockId, asValues = false).asInstanceOf[Option[ByteBuffer]] - } + } private def doGetRemote(blockId: BlockId, asValues: Boolean): Option[Any] = { require(blockId != null, "BlockId is null") @@ -447,9 +471,8 @@ private[spark] class BlockManager( * so that we can control the maxMegabytesInFlight for the fetch. */ def getMultiple( - blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])], serializer: Serializer) - : BlockFetcherIterator = { - + blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])], + serializer: Serializer): BlockFetcherIterator = { val iter = if (conf.getBoolean("spark.shuffle.use.netty", false)) { new BlockFetcherIterator.NettyBlockFetcherIterator(this, blocksByAddress, serializer) @@ -461,8 +484,11 @@ private[spark] class BlockManager( iter } - def put(blockId: BlockId, values: Iterator[Any], level: StorageLevel, tellMaster: Boolean) - : Long = { + def put( + blockId: BlockId, + values: Iterator[Any], + level: StorageLevel, + tellMaster: Boolean): Seq[(BlockId, BlockStatus)] = { doPut(blockId, IteratorValues(values), level, tellMaster) } @@ -472,41 +498,58 @@ private[spark] class BlockManager( * This is currently used for writing shuffle files out. Callers should handle error * cases. */ - def getDiskWriter(blockId: BlockId, file: File, serializer: Serializer, bufferSize: Int) - : BlockObjectWriter = { + def getDiskWriter( + blockId: BlockId, + file: File, + serializer: Serializer, + bufferSize: Int): BlockObjectWriter = { val compressStream: OutputStream => OutputStream = wrapForCompression(blockId, _) val syncWrites = conf.getBoolean("spark.shuffle.sync", false) new DiskBlockObjectWriter(blockId, file, serializer, bufferSize, compressStream, syncWrites) } /** - * Put a new block of values to the block manager. Returns its (estimated) size in bytes. + * Put a new block of values to the block manager. Return a list of blocks updated as a + * result of this put. */ - def put(blockId: BlockId, values: ArrayBuffer[Any], level: StorageLevel, - tellMaster: Boolean = true) : Long = { + def put( + blockId: BlockId, + values: ArrayBuffer[Any], + level: StorageLevel, + tellMaster: Boolean = true): Seq[(BlockId, BlockStatus)] = { require(values != null, "Values is null") doPut(blockId, ArrayBufferValues(values), level, tellMaster) } /** - * Put a new block of serialized bytes to the block manager. + * Put a new block of serialized bytes to the block manager. Return a list of blocks updated + * as a result of this put. */ - def putBytes(blockId: BlockId, bytes: ByteBuffer, level: StorageLevel, - tellMaster: Boolean = true) { + def putBytes( + blockId: BlockId, + bytes: ByteBuffer, + level: StorageLevel, + tellMaster: Boolean = true): Seq[(BlockId, BlockStatus)] = { require(bytes != null, "Bytes is null") doPut(blockId, ByteBufferValues(bytes), level, tellMaster) } - private def doPut(blockId: BlockId, - data: Values, - level: StorageLevel, tellMaster: Boolean = true): Long = { + private def doPut( + blockId: BlockId, + data: Values, + level: StorageLevel, + tellMaster: Boolean = true): Seq[(BlockId, BlockStatus)] = { + require(blockId != null, "BlockId is null") require(level != null && level.isValid, "StorageLevel is null or invalid") + // Return value + val updatedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] + // Remember the block's storage level so that we can correctly drop it to disk if it needs // to be dropped right after it got put into memory. Note, however, that other threads will // not be able to get() this block until we call markReady on its BlockInfo. - val myInfo = { + val putBlockInfo = { val tinfo = new BlockInfo(level, tellMaster) // Do atomically ! val oldBlockOpt = blockInfo.putIfAbsent(blockId, tinfo) @@ -514,7 +557,7 @@ private[spark] class BlockManager( if (oldBlockOpt.isDefined) { if (oldBlockOpt.get.waitForReady()) { logWarning("Block " + blockId + " already exists on this machine; not re-adding it") - return oldBlockOpt.get.size + return updatedBlocks } // TODO: So the block info exists - but previous attempt to load it (?) failed. @@ -536,7 +579,7 @@ private[spark] class BlockManager( // Ditto for the bytes after the put var bytesAfterPut: ByteBuffer = null - // Size of the block in bytes (to return to caller) + // Size of the block in bytes var size = 0L // If we're storing bytes, then initiate the replication before storing them locally. @@ -551,7 +594,7 @@ private[spark] class BlockManager( null } - myInfo.synchronized { + putBlockInfo.synchronized { logTrace("Put for block " + blockId + " took " + Utils.getUsedTimeMs(startTimeMs) + " to get into synchronized block") @@ -566,7 +609,7 @@ private[spark] class BlockManager( case ArrayBufferValues(array) => memoryStore.putValues(blockId, array, level, true) case ByteBufferValues(bytes) => { - bytes.rewind(); + bytes.rewind() memoryStore.putBytes(blockId, bytes, level) } } @@ -575,6 +618,8 @@ private[spark] class BlockManager( case Right(newBytes) => bytesAfterPut = newBytes case Left(newIterator) => valuesAfterPut = newIterator } + // Keep track of which blocks are dropped from memory + res.droppedBlocks.foreach { block => updatedBlocks += block } } else { // Save directly to disk. // Don't get back the bytes unless we replicate them. @@ -586,7 +631,7 @@ private[spark] class BlockManager( case ArrayBufferValues(array) => diskStore.putValues(blockId, array, level, askForBytes) case ByteBufferValues(bytes) => { - bytes.rewind(); + bytes.rewind() diskStore.putBytes(blockId, bytes, level) } } @@ -597,21 +642,25 @@ private[spark] class BlockManager( } } - // Now that the block is in either the memory or disk store, let other threads read it, - // and tell the master about it. - marked = true - myInfo.markReady(size) - if (tellMaster) { - reportBlockStatus(blockId, myInfo) + val putBlockStatus = getCurrentBlockStatus(blockId, putBlockInfo) + if (putBlockStatus.storageLevel != StorageLevel.NONE) { + // Now that the block is in either the memory or disk store, let other threads read it, + // and tell the master about it. + marked = true + putBlockInfo.markReady(size) + if (tellMaster) { + reportBlockStatus(blockId, putBlockInfo, putBlockStatus) + } + updatedBlocks += ((blockId, putBlockStatus)) } } finally { - // If we failed at putting the block to memory/disk, notify other possible readers + // If we failed in putting the block to memory/disk, notify other possible readers // that it has failed, and then remove it from the block info map. - if (! marked) { + if (!marked) { // Note that the remove must happen before markFailure otherwise another thread // could've inserted a new BlockInfo before we remove it. blockInfo.remove(blockId) - myInfo.markFailure() + putBlockInfo.markFailure() logWarning("Putting block " + blockId + " failed") } } @@ -650,7 +699,7 @@ private[spark] class BlockManager( Utils.getUsedTimeMs(startTimeMs)) } - size + updatedBlocks } /** @@ -687,28 +736,42 @@ private[spark] class BlockManager( /** * Write a block consisting of a single object. */ - def putSingle(blockId: BlockId, value: Any, level: StorageLevel, tellMaster: Boolean = true) { + def putSingle( + blockId: BlockId, + value: Any, + level: StorageLevel, + tellMaster: Boolean = true): Seq[(BlockId, BlockStatus)] = { put(blockId, Iterator(value), level, tellMaster) } /** * Drop a block from memory, possibly putting it on disk if applicable. Called when the memory * store reaches its limit and needs to free up space. + * + * Return the block status if the given block has been updated, else None. */ - def dropFromMemory(blockId: BlockId, data: Either[ArrayBuffer[Any], ByteBuffer]) { + def dropFromMemory( + blockId: BlockId, + data: Either[ArrayBuffer[Any], ByteBuffer]): Option[BlockStatus] = { + logInfo("Dropping block " + blockId + " from memory") val info = blockInfo.get(blockId).orNull + + // If the block has not already been dropped if (info != null) { info.synchronized { // required ? As of now, this will be invoked only for blocks which are ready // But in case this changes in future, adding for consistency sake. - if (! info.waitForReady() ) { + if (!info.waitForReady()) { // If we get here, the block write failed. logWarning("Block " + blockId + " was marked as failure. Nothing to drop") - return + return None } + var blockIsUpdated = false val level = info.level + + // Drop to disk, if storage level requires if (level.useDisk && !diskStore.contains(blockId)) { logInfo("Writing block " + blockId + " to disk") data match { @@ -717,24 +780,33 @@ private[spark] class BlockManager( case Right(bytes) => diskStore.putBytes(blockId, bytes, level) } + blockIsUpdated = true } + + // Actually drop from memory store val droppedMemorySize = if (memoryStore.contains(blockId)) memoryStore.getSize(blockId) else 0L - val blockWasRemoved = memoryStore.remove(blockId) - if (!blockWasRemoved) { + val blockIsRemoved = memoryStore.remove(blockId) + if (blockIsRemoved) { + blockIsUpdated = true + } else { logWarning("Block " + blockId + " could not be dropped from memory as it does not exist") } + + val status = getCurrentBlockStatus(blockId, info) if (info.tellMaster) { - reportBlockStatus(blockId, info, droppedMemorySize) + reportBlockStatus(blockId, info, status, droppedMemorySize) } if (!level.useDisk) { // The block is completely gone from this node; forget it so we can put() it again later. blockInfo.remove(blockId) } + if (blockIsUpdated) { + return Some(status) + } } - } else { - // The block has already been dropped } + None } /** @@ -766,7 +838,8 @@ private[spark] class BlockManager( } blockInfo.remove(blockId) if (tellMaster && info.tellMaster) { - reportBlockStatus(blockId, info) + val status = getCurrentBlockStatus(blockId, info) + reportBlockStatus(blockId, info, status) } } else { // The block has already been removed; do nothing. @@ -801,7 +874,8 @@ private[spark] class BlockManager( iterator.remove() logInfo("Dropped block " + id) } - reportBlockStatus(id, info) + val status = getCurrentBlockStatus(id, info) + reportBlockStatus(id, info, status) } } } @@ -911,9 +985,8 @@ private[spark] object BlockManager extends Logging { def blockIdsToBlockManagers( blockIds: Array[BlockId], env: SparkEnv, - blockManagerMaster: BlockManagerMaster = null) - : Map[BlockId, Seq[BlockManagerId]] = - { + blockManagerMaster: BlockManagerMaster = null): Map[BlockId, Seq[BlockManagerId]] = { + // blockManagerMaster != null is used in tests assert (env != null || blockManagerMaster != null) val blockLocations: Seq[Seq[BlockManagerId]] = if (blockManagerMaster == null) { @@ -932,18 +1005,14 @@ private[spark] object BlockManager extends Logging { def blockIdsToExecutorIds( blockIds: Array[BlockId], env: SparkEnv, - blockManagerMaster: BlockManagerMaster = null) - : Map[BlockId, Seq[String]] = - { + blockManagerMaster: BlockManagerMaster = null): Map[BlockId, Seq[String]] = { blockIdsToBlockManagers(blockIds, env, blockManagerMaster).mapValues(s => s.map(_.executorId)) } def blockIdsToHosts( blockIds: Array[BlockId], env: SparkEnv, - blockManagerMaster: BlockManagerMaster = null) - : Map[BlockId, Seq[String]] = - { + blockManagerMaster: BlockManagerMaster = null): Map[BlockId, Seq[String]] = { blockIdsToBlockManagers(blockIds, env, blockManagerMaster).mapValues(s => s.map(_.host)) } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala index 98cd6e68fa724..be537d77309bc 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala @@ -50,7 +50,6 @@ private[spark] class BlockManagerId private ( // DEBUG code Utils.checkHost(host) assert (port > 0) - host + ":" + port } @@ -93,7 +92,7 @@ private[spark] class BlockManagerId private ( private[spark] object BlockManagerId { /** - * Returns a [[org.apache.spark.storage.BlockManagerId]] for the given configuraiton. + * Returns a [[org.apache.spark.storage.BlockManagerId]] for the given configuration. * * @param execId ID of the executor. * @param host Host name of the block manager. diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index e531467cccb40..ed6937851b836 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -28,8 +28,7 @@ import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util.AkkaUtils private[spark] -class BlockManagerMaster(var driverActor : ActorRef, conf: SparkConf) extends Logging { - +class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Logging { val AKKA_RETRY_ATTEMPTS: Int = conf.getInt("spark.akka.num.retries", 3) val AKKA_RETRY_INTERVAL_MS: Int = conf.getInt("spark.akka.retry.wait", 3000) @@ -53,8 +52,7 @@ class BlockManagerMaster(var driverActor : ActorRef, conf: SparkConf) extends Lo } /** Register the BlockManager's id with the driver. */ - def registerBlockManager( - blockManagerId: BlockManagerId, maxMemSize: Long, slaveActor: ActorRef) { + def registerBlockManager(blockManagerId: BlockManagerId, maxMemSize: Long, slaveActor: ActorRef) { logInfo("Trying to register BlockManager") tell(RegisterBlockManager(blockManagerId, maxMemSize, slaveActor)) logInfo("Registered BlockManager") diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index a999d76a326a6..ff2652b640272 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -28,6 +28,7 @@ import akka.actor.{Actor, ActorRef, Cancellable} import akka.pattern.ask import org.apache.spark.{Logging, SparkConf, SparkException} +import org.apache.spark.scheduler._ import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util.{AkkaUtils, Utils} @@ -36,11 +37,11 @@ import org.apache.spark.util.{AkkaUtils, Utils} * all slaves' block managers. */ private[spark] -class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Actor with Logging { +class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus: LiveListenerBus) + extends Actor with Logging { // Mapping from block manager id to the block manager's information. - private val blockManagerInfo = - new mutable.HashMap[BlockManagerId, BlockManagerMasterActor.BlockManagerInfo] + private val blockManagerInfo = new mutable.HashMap[BlockManagerId, BlockManagerInfo] // Mapping from executor ID to block manager ID. private val blockManagerIdByExecutor = new mutable.HashMap[String, BlockManagerId] @@ -160,6 +161,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act blockLocations.remove(locations) } } + listenerBus.post(SparkListenerBlockManagerRemoved(blockManagerId)) } private def expireDeadHosts() { @@ -217,8 +219,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act private def storageStatus: Array[StorageStatus] = { blockManagerInfo.map { case(blockManagerId, info) => - import collection.JavaConverters._ - StorageStatus(blockManagerId, info.maxMem, info.blocks.asScala.toMap) + val blockMap = mutable.Map[BlockId, BlockStatus](info.blocks.toSeq: _*) + new StorageStatus(blockManagerId, info.maxMem, blockMap) }.toArray } @@ -233,9 +235,10 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act case None => blockManagerIdByExecutor(id.executorId) = id } - blockManagerInfo(id) = new BlockManagerMasterActor.BlockManagerInfo( - id, System.currentTimeMillis(), maxMemSize, slaveActor) + blockManagerInfo(id) = + new BlockManagerInfo(id, System.currentTimeMillis(), maxMemSize, slaveActor) } + listenerBus.post(SparkListenerBlockManagerAdded(id, maxMemSize)) } private def updateBlockInfo( @@ -307,97 +310,96 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act } -private[spark] -object BlockManagerMasterActor { - - case class BlockStatus(storageLevel: StorageLevel, memSize: Long, diskSize: Long) +private[spark] case class BlockStatus(storageLevel: StorageLevel, memSize: Long, diskSize: Long) - class BlockManagerInfo( - val blockManagerId: BlockManagerId, - timeMs: Long, - val maxMem: Long, - val slaveActor: ActorRef) - extends Logging { +private[spark] class BlockManagerInfo( + val blockManagerId: BlockManagerId, + timeMs: Long, + val maxMem: Long, + val slaveActor: ActorRef) + extends Logging { - private var _lastSeenMs: Long = timeMs - private var _remainingMem: Long = maxMem + private var _lastSeenMs: Long = timeMs + private var _remainingMem: Long = maxMem - // Mapping from block id to its status. - private val _blocks = new JHashMap[BlockId, BlockStatus] + // Mapping from block id to its status. + private val _blocks = new JHashMap[BlockId, BlockStatus] - logInfo("Registering block manager %s with %s RAM".format( - blockManagerId.hostPort, Utils.bytesToString(maxMem))) + logInfo("Registering block manager %s with %s RAM".format( + blockManagerId.hostPort, Utils.bytesToString(maxMem))) - def updateLastSeenMs() { - _lastSeenMs = System.currentTimeMillis() - } + def updateLastSeenMs() { + _lastSeenMs = System.currentTimeMillis() + } - def updateBlockInfo(blockId: BlockId, storageLevel: StorageLevel, memSize: Long, - diskSize: Long) { + def updateBlockInfo( + blockId: BlockId, + storageLevel: StorageLevel, + memSize: Long, + diskSize: Long) { - updateLastSeenMs() + updateLastSeenMs() - if (_blocks.containsKey(blockId)) { - // The block exists on the slave already. - val originalLevel: StorageLevel = _blocks.get(blockId).storageLevel + if (_blocks.containsKey(blockId)) { + // The block exists on the slave already. + val originalLevel: StorageLevel = _blocks.get(blockId).storageLevel - if (originalLevel.useMemory) { - _remainingMem += memSize - } + if (originalLevel.useMemory) { + _remainingMem += memSize } + } - if (storageLevel.isValid) { - // isValid means it is either stored in-memory or on-disk. - // But the memSize here indicates the data size in or dropped from memory, - // and the diskSize here indicates the data size in or dropped to disk. - // They can be both larger than 0, when a block is dropped from memory to disk. - // Therefore, a safe way to set BlockStatus is to set its info in accurate modes. - if (storageLevel.useMemory) { - _blocks.put(blockId, BlockStatus(storageLevel, memSize, 0)) - _remainingMem -= memSize - logInfo("Added %s in memory on %s (size: %s, free: %s)".format( - blockId, blockManagerId.hostPort, Utils.bytesToString(memSize), - Utils.bytesToString(_remainingMem))) - } - if (storageLevel.useDisk) { - _blocks.put(blockId, BlockStatus(storageLevel, 0, diskSize)) - logInfo("Added %s on disk on %s (size: %s)".format( - blockId, blockManagerId.hostPort, Utils.bytesToString(diskSize))) - } - } else if (_blocks.containsKey(blockId)) { - // If isValid is not true, drop the block. - val blockStatus: BlockStatus = _blocks.get(blockId) - _blocks.remove(blockId) - if (blockStatus.storageLevel.useMemory) { - _remainingMem += blockStatus.memSize - logInfo("Removed %s on %s in memory (size: %s, free: %s)".format( - blockId, blockManagerId.hostPort, Utils.bytesToString(blockStatus.memSize), - Utils.bytesToString(_remainingMem))) - } - if (blockStatus.storageLevel.useDisk) { - logInfo("Removed %s on %s on disk (size: %s)".format( - blockId, blockManagerId.hostPort, Utils.bytesToString(blockStatus.diskSize))) - } + if (storageLevel.isValid) { + /* isValid means it is either stored in-memory or on-disk. + * But the memSize here indicates the data size in or dropped from memory, + * and the diskSize here indicates the data size in or dropped to disk. + * They can be both larger than 0, when a block is dropped from memory to disk. + * Therefore, a safe way to set BlockStatus is to set its info in accurate modes. */ + if (storageLevel.useMemory) { + _blocks.put(blockId, BlockStatus(storageLevel, memSize, 0)) + _remainingMem -= memSize + logInfo("Added %s in memory on %s (size: %s, free: %s)".format( + blockId, blockManagerId.hostPort, Utils.bytesToString(memSize), + Utils.bytesToString(_remainingMem))) + } + if (storageLevel.useDisk) { + _blocks.put(blockId, BlockStatus(storageLevel, 0, diskSize)) + logInfo("Added %s on disk on %s (size: %s)".format( + blockId, blockManagerId.hostPort, Utils.bytesToString(diskSize))) + } + } else if (_blocks.containsKey(blockId)) { + // If isValid is not true, drop the block. + val blockStatus: BlockStatus = _blocks.get(blockId) + _blocks.remove(blockId) + if (blockStatus.storageLevel.useMemory) { + _remainingMem += blockStatus.memSize + logInfo("Removed %s on %s in memory (size: %s, free: %s)".format( + blockId, blockManagerId.hostPort, Utils.bytesToString(blockStatus.memSize), + Utils.bytesToString(_remainingMem))) + } + if (blockStatus.storageLevel.useDisk) { + logInfo("Removed %s on %s on disk (size: %s)".format( + blockId, blockManagerId.hostPort, Utils.bytesToString(blockStatus.diskSize))) } } + } - def removeBlock(blockId: BlockId) { - if (_blocks.containsKey(blockId)) { - _remainingMem += _blocks.get(blockId).memSize - _blocks.remove(blockId) - } + def removeBlock(blockId: BlockId) { + if (_blocks.containsKey(blockId)) { + _remainingMem += _blocks.get(blockId).memSize + _blocks.remove(blockId) } + } - def remainingMem: Long = _remainingMem + def remainingMem: Long = _remainingMem - def lastSeenMs: Long = _lastSeenMs + def lastSeenMs: Long = _lastSeenMs - def blocks: JHashMap[BlockId, BlockStatus] = _blocks + def blocks: JHashMap[BlockId, BlockStatus] = _blocks - override def toString: String = "BlockManagerInfo " + timeMs + " " + _remainingMem + override def toString: String = "BlockManagerInfo " + timeMs + " " + _remainingMem - def clear() { - _blocks.clear() - } + def clear() { + _blocks.clear() } } diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 38836d44b04e8..488f1ea9628f5 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -49,7 +49,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) } } - override def putBytes(blockId: BlockId, _bytes: ByteBuffer, level: StorageLevel) : PutResult = { + override def putBytes(blockId: BlockId, _bytes: ByteBuffer, level: StorageLevel): PutResult = { // Work on a duplicate - since the original input might be used elsewhere. val bytes = _bytes.duplicate() bytes.rewind() @@ -70,16 +70,15 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) blockId: BlockId, values: ArrayBuffer[Any], level: StorageLevel, - returnValues: Boolean) - : PutResult = { + returnValues: Boolean): PutResult = { if (level.deserialized) { val sizeEstimate = SizeEstimator.estimate(values.asInstanceOf[AnyRef]) - tryToPut(blockId, values, sizeEstimate, true) - PutResult(sizeEstimate, Left(values.toIterator)) + val putAttempt = tryToPut(blockId, values, sizeEstimate, deserialized = true) + PutResult(sizeEstimate, Left(values.iterator), putAttempt.droppedBlocks) } else { - val bytes = blockManager.dataSerialize(blockId, values.toIterator) - tryToPut(blockId, bytes, bytes.limit, false) - PutResult(bytes.limit(), Right(bytes.duplicate())) + val bytes = blockManager.dataSerialize(blockId, values.iterator) + val putAttempt = tryToPut(blockId, bytes, bytes.limit, deserialized = false) + PutResult(bytes.limit(), Right(bytes.duplicate()), putAttempt.droppedBlocks) } } @@ -87,20 +86,10 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) blockId: BlockId, values: Iterator[Any], level: StorageLevel, - returnValues: Boolean) - : PutResult = { - - if (level.deserialized) { - val valueEntries = new ArrayBuffer[Any]() - valueEntries ++= values - val sizeEstimate = SizeEstimator.estimate(valueEntries.asInstanceOf[AnyRef]) - tryToPut(blockId, valueEntries, sizeEstimate, true) - PutResult(sizeEstimate, Left(valueEntries.toIterator)) - } else { - val bytes = blockManager.dataSerialize(blockId, values) - tryToPut(blockId, bytes, bytes.limit, false) - PutResult(bytes.limit(), Right(bytes.duplicate())) - } + returnValues: Boolean): PutResult = { + val valueEntries = new ArrayBuffer[Any]() + valueEntries ++= values + putValues(blockId, valueEntries, level, returnValues) } override def getBytes(blockId: BlockId): Option[ByteBuffer] = { @@ -164,19 +153,34 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) * an ArrayBuffer if deserialized is true or a ByteBuffer otherwise. Its (possibly estimated) * size must also be passed by the caller. * - * Locks on the object putLock to ensure that all the put requests and its associated block + * Lock on the object putLock to ensure that all the put requests and its associated block * dropping is done by only on thread at a time. Otherwise while one thread is dropping * blocks to free memory for one block, another thread may use up the freed space for * another block. + * + * Return whether put was successful, along with the blocks dropped in the process. */ - private def tryToPut(blockId: BlockId, value: Any, size: Long, deserialized: Boolean): Boolean = { - // TODO: Its possible to optimize the locking by locking entries only when selecting blocks - // to be dropped. Once the to-be-dropped blocks have been selected, and lock on entries has been - // released, it must be ensured that those to-be-dropped blocks are not double counted for - // freeing up more space for another block that needs to be put. Only then the actually dropping - // of blocks (and writing to disk if necessary) can proceed in parallel. + private def tryToPut( + blockId: BlockId, + value: Any, + size: Long, + deserialized: Boolean): ResultWithDroppedBlocks = { + + /* TODO: Its possible to optimize the locking by locking entries only when selecting blocks + * to be dropped. Once the to-be-dropped blocks have been selected, and lock on entries has + * been released, it must be ensured that those to-be-dropped blocks are not double counted + * for freeing up more space for another block that needs to be put. Only then the actually + * dropping of blocks (and writing to disk if necessary) can proceed in parallel. */ + + var putSuccess = false + val droppedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] + putLock.synchronized { - if (ensureFreeSpace(blockId, size)) { + val freeSpaceResult = ensureFreeSpace(blockId, size) + val enoughFreeSpace = freeSpaceResult.success + droppedBlocks ++= freeSpaceResult.droppedBlocks + + if (enoughFreeSpace) { val entry = new Entry(value, size, deserialized) entries.synchronized { entries.put(blockId, entry) @@ -189,7 +193,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) logInfo("Block %s stored as bytes to memory (size %s, free %s)".format( blockId, Utils.bytesToString(size), Utils.bytesToString(freeMemory))) } - true + putSuccess = true } else { // Tell the block manager that we couldn't put it in memory so that it can drop it to // disk if the block allows disk storage. @@ -198,29 +202,33 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) } else { Right(value.asInstanceOf[ByteBuffer].duplicate()) } - blockManager.dropFromMemory(blockId, data) - false + val droppedBlockStatus = blockManager.dropFromMemory(blockId, data) + droppedBlockStatus.foreach { status => droppedBlocks += ((blockId, status)) } } } + ResultWithDroppedBlocks(putSuccess, droppedBlocks) } /** - * Tries to free up a given amount of space to store a particular block, but can fail and return - * false if either the block is bigger than our memory or it would require replacing another - * block from the same RDD (which leads to a wasteful cyclic replacement pattern for RDDs that + * Try to free up a given amount of space to store a particular block, but can fail if + * either the block is bigger than our memory or it would require replacing another block + * from the same RDD (which leads to a wasteful cyclic replacement pattern for RDDs that * don't fit into memory that we want to avoid). * - * Assumes that a lock is held by the caller to ensure only one thread is dropping blocks. + * Assume that a lock is held by the caller to ensure only one thread is dropping blocks. * Otherwise, the freed space may fill up before the caller puts in their new value. + * + * Return whether there is enough free space, along with the blocks dropped in the process. */ - private def ensureFreeSpace(blockIdToAdd: BlockId, space: Long): Boolean = { - + private def ensureFreeSpace(blockIdToAdd: BlockId, space: Long): ResultWithDroppedBlocks = { logInfo("ensureFreeSpace(%d) called with curMem=%d, maxMem=%d".format( space, currentMemory, maxMemory)) + val droppedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] + if (space > maxMemory) { logInfo("Will not store " + blockIdToAdd + " as it is larger than our memory limit") - return false + return ResultWithDroppedBlocks(success = false, droppedBlocks) } if (maxMemory - currentMemory < space) { @@ -256,17 +264,18 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) } else { Right(entry.value.asInstanceOf[ByteBuffer].duplicate()) } - blockManager.dropFromMemory(blockId, data) + val droppedBlockStatus = blockManager.dropFromMemory(blockId, data) + droppedBlockStatus.foreach { status => droppedBlocks += ((blockId, status)) } } } - return true + return ResultWithDroppedBlocks(success = true, droppedBlocks) } else { logInfo(s"Will not store $blockIdToAdd as it would require dropping another block " + "from the same RDD") - return false + return ResultWithDroppedBlocks(success = false, droppedBlocks) } } - true + ResultWithDroppedBlocks(success = true, droppedBlocks) } override def contains(blockId: BlockId): Boolean = { @@ -274,3 +283,6 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) } } +private case class ResultWithDroppedBlocks( + success: Boolean, + droppedBlocks: Seq[(BlockId, BlockStatus)]) diff --git a/core/src/main/scala/org/apache/spark/storage/PutResult.scala b/core/src/main/scala/org/apache/spark/storage/PutResult.scala index 2eba2f06b5bfd..f0eac7594ecf6 100644 --- a/core/src/main/scala/org/apache/spark/storage/PutResult.scala +++ b/core/src/main/scala/org/apache/spark/storage/PutResult.scala @@ -20,7 +20,13 @@ package org.apache.spark.storage import java.nio.ByteBuffer /** - * Result of adding a block into a BlockStore. Contains its estimated size, and possibly the - * values put if the caller asked for them to be returned (e.g. for chaining replication) + * Result of adding a block into a BlockStore. This case class contains a few things: + * (1) The estimated size of the put, + * (2) The values put if the caller asked for them to be returned (e.g. for chaining + * replication), and + * (3) A list of blocks dropped as a result of this put. This is always empty for DiskStore. */ -private[spark] case class PutResult(size: Long, data: Either[Iterator[_], ByteBuffer]) +private[spark] case class PutResult( + size: Long, + data: Either[Iterator[_], ByteBuffer], + droppedBlocks: Seq[(BlockId, BlockStatus)] = Seq.empty) diff --git a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala new file mode 100644 index 0000000000000..26565f56ad858 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import scala.collection.mutable + +import org.apache.spark.scheduler._ + +/** + * A SparkListener that maintains executor storage status + */ +private[spark] class StorageStatusListener extends SparkListener { + private val executorIdToStorageStatus = mutable.Map[String, StorageStatus]() + + def storageStatusList = executorIdToStorageStatus.values.toSeq + + /** Update storage status list to reflect updated block statuses */ + def updateStorageStatus(execId: String, updatedBlocks: Seq[(BlockId, BlockStatus)]) { + val filteredStatus = storageStatusList.find(_.blockManagerId.executorId == execId) + filteredStatus.foreach { storageStatus => + updatedBlocks.foreach { case (blockId, updatedStatus) => + storageStatus.blocks(blockId) = updatedStatus + } + } + } + + /** Update storage status list to reflect the removal of an RDD from the cache */ + def updateStorageStatus(unpersistedRDDId: Int) { + storageStatusList.foreach { storageStatus => + val unpersistedBlocksIds = storageStatus.rddBlocks.keys.filter(_.rddId == unpersistedRDDId) + unpersistedBlocksIds.foreach { blockId => + storageStatus.blocks(blockId) = BlockStatus(StorageLevel.NONE, 0L, 0L) + } + } + } + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { + val info = taskEnd.taskInfo + val metrics = taskEnd.taskMetrics + if (info != null && metrics != null) { + val execId = formatExecutorId(info.executorId) + val updatedBlocks = metrics.updatedBlocks.getOrElse(Seq[(BlockId, BlockStatus)]()) + if (updatedBlocks.length > 0) { + updateStorageStatus(execId, updatedBlocks) + } + } + } + + override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD) = synchronized { + updateStorageStatus(unpersistRDD.rddId) + } + + override def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded) { + synchronized { + val blockManagerId = blockManagerAdded.blockManagerId + val executorId = blockManagerId.executorId + val maxMem = blockManagerAdded.maxMem + val storageStatus = new StorageStatus(blockManagerId, maxMem) + executorIdToStorageStatus(executorId) = storageStatus + } + } + + override def onBlockManagerRemoved(blockManagerRemoved: SparkListenerBlockManagerRemoved) { + synchronized { + val executorId = blockManagerRemoved.blockManagerId.executorId + executorIdToStorageStatus.remove(executorId) + } + } + + /** + * In the local mode, there is a discrepancy between the executor ID according to the + * task ("localhost") and that according to SparkEnv (""). In the UI, this + * results in duplicate rows for the same executor. Thus, in this mode, we aggregate + * these two rows and use the executor ID of "" to be consistent. + */ + def formatExecutorId(execId: String): String = { + if (execId == "localhost") "" else execId + } +} diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 2d88a40fbb3f2..6153dfe0b7e13 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -17,13 +17,17 @@ package org.apache.spark.storage +import scala.collection.Map +import scala.collection.mutable + import org.apache.spark.SparkContext -import org.apache.spark.storage.BlockManagerMasterActor.BlockStatus import org.apache.spark.util.Utils private[spark] -case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long, - blocks: Map[BlockId, BlockStatus]) { +class StorageStatus( + val blockManagerId: BlockManagerId, + val maxMem: Long, + val blocks: mutable.Map[BlockId, BlockStatus] = mutable.Map.empty) { def memUsed() = blocks.values.map(_.memSize).reduceOption(_ + _).getOrElse(0L) @@ -43,14 +47,18 @@ case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long, } } -case class RDDInfo(id: Int, name: String, storageLevel: StorageLevel, - numCachedPartitions: Int, numPartitions: Int, memSize: Long, diskSize: Long) +private[spark] +class RDDInfo(val id: Int, val name: String, val numPartitions: Int, val storageLevel: StorageLevel) extends Ordered[RDDInfo] { + + var numCachedPartitions = 0 + var memSize = 0L + var diskSize = 0L + override def toString = { - import Utils.bytesToString ("RDD \"%s\" (%d) Storage: %s; CachedPartitions: %d; TotalPartitions: %d; MemorySize: %s; " + "DiskSize: %s").format(name, id, storageLevel.toString, numCachedPartitions, - numPartitions, bytesToString(memSize), bytesToString(diskSize)) + numPartitions, Utils.bytesToString(memSize), Utils.bytesToString(diskSize)) } override def compare(that: RDDInfo) = { @@ -62,55 +70,76 @@ case class RDDInfo(id: Int, name: String, storageLevel: StorageLevel, private[spark] object StorageUtils { - /* Returns RDD-level information, compiled from a list of StorageStatus objects */ - def rddInfoFromStorageStatus(storageStatusList: Seq[StorageStatus], - sc: SparkContext) : Array[RDDInfo] = { - rddInfoFromBlockStatusList( - storageStatusList.flatMap(_.rddBlocks).toMap[RDDBlockId, BlockStatus], sc) + /** + * Returns basic information of all RDDs persisted in the given SparkContext. This does not + * include storage information. + */ + def rddInfoFromSparkContext(sc: SparkContext): Array[RDDInfo] = { + sc.persistentRdds.values.map { rdd => + val rddName = Option(rdd.name).getOrElse(rdd.id.toString) + val rddNumPartitions = rdd.partitions.size + val rddStorageLevel = rdd.getStorageLevel + val rddInfo = new RDDInfo(rdd.id, rddName, rddNumPartitions, rddStorageLevel) + rddInfo + }.toArray } - /* Returns a map of blocks to their locations, compiled from a list of StorageStatus objects */ - def blockLocationsFromStorageStatus(storageStatusList: Seq[StorageStatus]) = { - val blockLocationPairs = storageStatusList - .flatMap(s => s.blocks.map(b => (b._1, s.blockManagerId.hostPort))) - blockLocationPairs.groupBy(_._1).map{case (k, v) => (k, v.unzip._2)}.toMap + /** Returns storage information of all RDDs persisted in the given SparkContext. */ + def rddInfoFromStorageStatus( + storageStatuses: Seq[StorageStatus], + sc: SparkContext): Array[RDDInfo] = { + rddInfoFromStorageStatus(storageStatuses, rddInfoFromSparkContext(sc)) } - /* Given a list of BlockStatus objets, returns information for each RDD */ - def rddInfoFromBlockStatusList(infos: Map[RDDBlockId, BlockStatus], - sc: SparkContext) : Array[RDDInfo] = { + /** Returns storage information of all RDDs in the given list. */ + def rddInfoFromStorageStatus( + storageStatuses: Seq[StorageStatus], + rddInfos: Seq[RDDInfo]): Array[RDDInfo] = { + + // Mapping from RDD ID -> an array of associated BlockStatuses + val blockStatusMap = storageStatuses.flatMap(_.rddBlocks).toMap + .groupBy { case (k, _) => k.rddId } + .mapValues(_.values.toArray) - // Group by rddId, ignore the partition name - val groupedRddBlocks = infos.groupBy { case(k, v) => k.rddId }.mapValues(_.values.toArray) + // Mapping from RDD ID -> the associated RDDInfo (with potentially outdated storage information) + val rddInfoMap = rddInfos.map { info => (info.id, info) }.toMap - // For each RDD, generate an RDDInfo object - val rddInfos = groupedRddBlocks.map { case (rddId, rddBlocks) => + val rddStorageInfos = blockStatusMap.flatMap { case (rddId, blocks) => // Add up memory and disk sizes - val memSize = rddBlocks.map(_.memSize).reduce(_ + _) - val diskSize = rddBlocks.map(_.diskSize).reduce(_ + _) - - // Get the friendly name and storage level for the RDD, if available - sc.persistentRdds.get(rddId).map { r => - val rddName = Option(r.name).getOrElse(rddId.toString) - val rddStorageLevel = r.getStorageLevel - RDDInfo(rddId, rddName, rddStorageLevel, rddBlocks.length, r.partitions.size, - memSize, diskSize) + val persistedBlocks = blocks.filter { status => status.memSize + status.diskSize > 0 } + val memSize = persistedBlocks.map(_.memSize).reduceOption(_ + _).getOrElse(0L) + val diskSize = persistedBlocks.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) + rddInfoMap.get(rddId).map { rddInfo => + rddInfo.numCachedPartitions = persistedBlocks.length + rddInfo.memSize = memSize + rddInfo.diskSize = diskSize + rddInfo } - }.flatten.toArray + }.toArray - scala.util.Sorting.quickSort(rddInfos) - - rddInfos + scala.util.Sorting.quickSort(rddStorageInfos) + rddStorageInfos } - /* Filters storage status by a given RDD id. */ - def filterStorageStatusByRDD(storageStatusList: Array[StorageStatus], rddId: Int) - : Array[StorageStatus] = { - - storageStatusList.map { status => - val newBlocks = status.rddBlocks.filterKeys(_.rddId == rddId).toMap[BlockId, BlockStatus] - //val newRemainingMem = status.maxMem - newBlocks.values.map(_.memSize).reduce(_ + _) - StorageStatus(status.blockManagerId, status.maxMem, newBlocks) + /** Returns a mapping from BlockId to the locations of the associated block. */ + def blockLocationsFromStorageStatus( + storageStatuses: Seq[StorageStatus]): Map[BlockId, Seq[String]] = { + val blockLocationPairs = storageStatuses.flatMap { storageStatus => + storageStatus.blocks.map { case (bid, _) => (bid, storageStatus.blockManagerId.hostPort) } } + blockLocationPairs.toMap + .groupBy { case (blockId, _) => blockId } + .mapValues(_.values.toSeq) + } + + /** Filters the given list of StorageStatus by the given RDD ID. */ + def filterStorageStatusByRDD( + storageStatuses: Seq[StorageStatus], + rddId: Int): Array[StorageStatus] = { + storageStatuses.map { status => + val filteredBlocks = status.rddBlocks.filterKeys(_.rddId == rddId).toSeq + val filteredBlockMap = mutable.Map[BlockId, BlockStatus](filteredBlocks: _*) + new StorageStatus(status.blockManagerId, status.maxMem, filteredBlockMap) + }.toArray } } diff --git a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala index 36f2a0fd02724..226ed2a132b00 100644 --- a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala @@ -22,9 +22,9 @@ import java.util.concurrent.ArrayBlockingQueue import akka.actor._ import util.Random -import org.apache.spark.SparkConf -import org.apache.spark.serializer.KryoSerializer import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.scheduler.LiveListenerBus +import org.apache.spark.serializer.KryoSerializer /** * This class tests the BlockManager and MemoryStore for thread safety and @@ -97,7 +97,8 @@ private[spark] object ThreadingTest { val conf = new SparkConf() val serializer = new KryoSerializer(conf) val blockManagerMaster = new BlockManagerMaster( - actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf))), conf) + actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf, new LiveListenerBus))), + conf) val blockManager = new BlockManager( "", actorSystem, blockManagerMaster, serializer, 1024 * 1024, conf, new SecurityManager(conf)) diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index e0555ca7ac02f..6e1736f6fbc23 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -17,30 +17,29 @@ package org.apache.spark.ui -import java.net.InetSocketAddress -import java.net.URL -import javax.servlet.http.{HttpServlet, HttpServletResponse, HttpServletRequest} +import java.net.{InetSocketAddress, URL} +import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} import scala.annotation.tailrec import scala.util.{Failure, Success, Try} import scala.xml.Node -import org.json4s.JValue -import org.json4s.jackson.JsonMethods.{pretty, render} - import org.eclipse.jetty.server.{DispatcherType, Server} -import org.eclipse.jetty.server.handler.HandlerList -import org.eclipse.jetty.servlet.{DefaultServlet, FilterHolder, ServletContextHandler, ServletHolder} +import org.eclipse.jetty.server.handler._ +import org.eclipse.jetty.servlet._ import org.eclipse.jetty.util.thread.QueuedThreadPool +import org.json4s.JValue +import org.json4s.jackson.JsonMethods.{pretty, render} import org.apache.spark.{Logging, SecurityManager, SparkConf} - -/** Utilities for launching a web server using Jetty's HTTP Server class */ +/** + * Utilities for launching a web server using Jetty's HTTP Server class + */ private[spark] object JettyUtils extends Logging { + // Base type for a function that returns something based on an HTTP request. Allows for // implicit conversion from many types of functions to jetty Handlers. - type Responder[T] = HttpServletRequest => T class ServletParams[T <% AnyRef](val responder: Responder[T], @@ -57,62 +56,73 @@ private[spark] object JettyUtils extends Logging { implicit def textResponderToServlet(responder: Responder[String]): ServletParams[String] = new ServletParams(responder, "text/plain") - def createServlet[T <% AnyRef](servletParams: ServletParams[T], + def createServlet[T <% AnyRef]( + servletParams: ServletParams[T], securityMgr: SecurityManager): HttpServlet = { new HttpServlet { - override def doGet(request: HttpServletRequest, - response: HttpServletResponse) { - if (securityMgr.checkUIViewPermissions(request.getRemoteUser())) { + override def doGet(request: HttpServletRequest, response: HttpServletResponse) { + if (securityMgr.checkUIViewPermissions(request.getRemoteUser)) { response.setContentType("%s;charset=utf-8".format(servletParams.contentType)) response.setStatus(HttpServletResponse.SC_OK) val result = servletParams.responder(request) response.setHeader("Cache-Control", "no-cache, no-store, must-revalidate") - response.getWriter().println(servletParams.extractFn(result)) + response.getWriter.println(servletParams.extractFn(result)) } else { response.setStatus(HttpServletResponse.SC_UNAUTHORIZED) response.setHeader("Cache-Control", "no-cache, no-store, must-revalidate") response.sendError(HttpServletResponse.SC_UNAUTHORIZED, - "User is not authorized to access this page."); + "User is not authorized to access this page.") } } } } - def createServletHandler(path: String, servlet: HttpServlet): ServletContextHandler = { - val contextHandler = new ServletContextHandler() + /** Create a context handler that responds to a request with the given path prefix */ + def createServletHandler[T <% AnyRef]( + path: String, + servletParams: ServletParams[T], + securityMgr: SecurityManager, + basePath: String = ""): ServletContextHandler = { + createServletHandler(path, createServlet(servletParams, securityMgr), basePath) + } + + /** Create a context handler that responds to a request with the given path prefix */ + def createServletHandler( + path: String, + servlet: HttpServlet, + basePath: String = ""): ServletContextHandler = { + val prefixedPath = attachPrefix(basePath, path) + val contextHandler = new ServletContextHandler val holder = new ServletHolder(servlet) - contextHandler.setContextPath(path) + contextHandler.setContextPath(prefixedPath) contextHandler.addServlet(holder, "/") contextHandler } - /** Creates a handler that always redirects the user to a given path */ - def createRedirectHandler(newPath: String, path: String): ServletContextHandler = { + /** Create a handler that always redirects the user to the given path */ + def createRedirectHandler( + srcPath: String, + destPath: String, + basePath: String = ""): ServletContextHandler = { + val prefixedDestPath = attachPrefix(basePath, destPath) val servlet = new HttpServlet { - override def doGet(request: HttpServletRequest, - response: HttpServletResponse) { - // make sure we don't end up with // in the middle - val newUri = new URL(new URL(request.getRequestURL.toString), newPath).toURI - response.sendRedirect(newUri.toString) + override def doGet(request: HttpServletRequest, response: HttpServletResponse) { + // Make sure we don't end up with "//" in the middle + val newUrl = new URL(new URL(request.getRequestURL.toString), prefixedDestPath).toString + response.sendRedirect(newUrl) } } - val contextHandler = new ServletContextHandler() - val holder = new ServletHolder(servlet) - contextHandler.setContextPath(path) - contextHandler.addServlet(holder, "/") - contextHandler + createServletHandler(srcPath, servlet, basePath) } - /** Creates a handler for serving files from a static directory */ + /** Create a handler for serving files from a static directory */ def createStaticHandler(resourceBase: String, path: String): ServletContextHandler = { - val contextHandler = new ServletContextHandler() + val contextHandler = new ServletContextHandler val staticHandler = new DefaultServlet val holder = new ServletHolder(staticHandler) Option(getClass.getClassLoader.getResource(resourceBase)) match { case Some(res) => holder.setInitParameter("resourceBase", res.toString) - holder.setInitParameter("welcomeServlets", "false") - holder.setInitParameter("pathInfoOnly", "false") case None => throw new Exception("Could not find resource path for Web UI: " + resourceBase) } @@ -121,6 +131,7 @@ private[spark] object JettyUtils extends Logging { contextHandler } + /** Add security filters, if any, do the given list of ServletContextHandlers */ private def addFilters(handlers: Seq[ServletContextHandler], conf: SparkConf) { val filters: Array[String] = conf.get("spark.ui.filters", "").split(',').map(_.trim()) filters.foreach { @@ -129,7 +140,7 @@ private[spark] object JettyUtils extends Logging { logInfo("Adding filter: " + filter) val holder : FilterHolder = new FilterHolder() holder.setClassName(filter) - // get any parameters for each filter + // Get any parameters for each filter val paramName = "spark." + filter + ".params" val params = conf.get(paramName, "").split(',').map(_.trim()).toSet params.foreach { @@ -147,18 +158,21 @@ private[spark] object JettyUtils extends Logging { } /** - * Attempts to start a Jetty server at the supplied hostName:port which uses the supplied - * handlers. + * Attempt to start a Jetty server bound to the supplied hostName:port using the given + * context handlers. * - * If the desired port number is contented, continues incrementing ports until a free port is - * found. Returns the chosen port and the jetty Server object. + * If the desired port number is contended, continues incrementing ports until a free port is + * found. Return the jetty Server object, the chosen port, and a mutable collection of handlers. */ - def startJettyServer(hostName: String, port: Int, handlers: Seq[ServletContextHandler], - conf: SparkConf): (Server, Int) = { - + def startJettyServer( + hostName: String, + port: Int, + handlers: Seq[ServletContextHandler], + conf: SparkConf): ServerInfo = { + + val collection = new ContextHandlerCollection + collection.setHandlers(handlers.toArray) addFilters(handlers, conf) - val handlerList = new HandlerList - handlerList.setHandlers(handlers.toArray) @tailrec def connect(currentPort: Int): (Server, Int) = { @@ -166,7 +180,7 @@ private[spark] object JettyUtils extends Logging { val pool = new QueuedThreadPool pool.setDaemon(true) server.setThreadPool(pool) - server.setHandler(handlerList) + server.setHandler(collection) Try { server.start() @@ -181,6 +195,17 @@ private[spark] object JettyUtils extends Logging { } } - connect(port) + val (server, boundPort) = connect(port) + ServerInfo(server, boundPort, collection) + } + + /** Attach a prefix to the given path, but avoid returning an empty path */ + private def attachPrefix(basePath: String, relativePath: String): String = { + if (basePath == "") relativePath else (basePath + relativePath).stripSuffix("/") } } + +private[spark] case class ServerInfo( + server: Server, + boundPort: Int, + rootHandler: ContextHandlerCollection) diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 5f0dee64fedb7..fd638c83aac6e 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -17,12 +17,11 @@ package org.apache.spark.ui -import javax.servlet.http.HttpServletRequest - -import org.eclipse.jetty.server.Server import org.eclipse.jetty.servlet.ServletContextHandler -import org.apache.spark.{Logging, SparkContext, SparkEnv} +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, SparkEnv} +import org.apache.spark.scheduler._ +import org.apache.spark.storage.StorageStatusListener import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.env.EnvironmentUI import org.apache.spark.ui.exec.ExecutorsUI @@ -31,34 +30,57 @@ import org.apache.spark.ui.storage.BlockManagerUI import org.apache.spark.util.Utils /** Top level user interface for Spark */ -private[spark] class SparkUI(sc: SparkContext) extends Logging { - val host = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(Utils.localHostName()) - val port = sc.conf.get("spark.ui.port", SparkUI.DEFAULT_PORT).toInt - var boundPort: Option[Int] = None - var server: Option[Server] = None - - val handlers = Seq[ServletContextHandler] ( - createStaticHandler(SparkUI.STATIC_RESOURCE_DIR + "/static", "/static"), - createRedirectHandler("/stages", "/") - ) - val storage = new BlockManagerUI(sc) - val jobs = new JobProgressUI(sc) - val env = new EnvironmentUI(sc) - val exec = new ExecutorsUI(sc) - - // Add MetricsServlet handlers by default - val metricsServletHandlers = SparkEnv.get.metricsSystem.getServletHandlers - - val allHandlers = storage.getHandlers ++ jobs.getHandlers ++ env.getHandlers ++ - exec.getHandlers ++ metricsServletHandlers ++ handlers +private[spark] class SparkUI( + val sc: SparkContext, + conf: SparkConf, + val listenerBus: SparkListenerBus, + val appName: String, + val basePath: String = "") + extends Logging { + + def this(sc: SparkContext) = this(sc, sc.conf, sc.listenerBus, sc.appName) + def this(conf: SparkConf, listenerBus: SparkListenerBus, appName: String, basePath: String) = + this(null, conf, listenerBus, appName, basePath) + + // If SparkContext is not provided, assume the associated application is not live + val live = sc != null + + val securityManager = if (live) sc.env.securityManager else new SecurityManager(conf) + + private val host = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(Utils.localHostName()) + private val port = conf.get("spark.ui.port", SparkUI.DEFAULT_PORT).toInt + private var serverInfo: Option[ServerInfo] = None + + private val storage = new BlockManagerUI(this) + private val jobs = new JobProgressUI(this) + private val env = new EnvironmentUI(this) + private val exec = new ExecutorsUI(this) + + val handlers: Seq[ServletContextHandler] = { + val metricsServletHandlers = if (live) { + SparkEnv.get.metricsSystem.getServletHandlers + } else { + Array[ServletContextHandler]() + } + storage.getHandlers ++ + jobs.getHandlers ++ + env.getHandlers ++ + exec.getHandlers ++ + metricsServletHandlers ++ + Seq[ServletContextHandler] ( + createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static"), + createRedirectHandler("/", "/stages", basePath) + ) + } + + // Maintain executor storage status through Spark events + val storageStatusListener = new StorageStatusListener /** Bind the HTTP server which backs this web interface */ def bind() { try { - val (srv, usedPort) = JettyUtils.startJettyServer(host, port, allHandlers, sc.conf) - logInfo("Started Spark Web UI at http://%s:%d".format(host, usedPort)) - server = Some(srv) - boundPort = Some(usedPort) + serverInfo = Some(startJettyServer(host, port, handlers, sc.conf)) + logInfo("Started Spark Web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => logError("Failed to create Spark JettyUtils", e) @@ -66,25 +88,34 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { } } + def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) + /** Initialize all components of the server */ def start() { - // NOTE: This is decoupled from bind() because of the following dependency cycle: - // DAGScheduler() requires that the port of this server is known - // This server must register all handlers, including JobProgressUI, before binding - // JobProgressUI registers a listener with SparkContext, which requires sc to initialize + storage.start() jobs.start() + env.start() exec.start() + + // Storage status listener must receive events first, as other listeners depend on its state + listenerBus.addListener(storageStatusListener) + listenerBus.addListener(storage.listener) + listenerBus.addListener(jobs.listener) + listenerBus.addListener(env.listener) + listenerBus.addListener(exec.listener) } def stop() { - server.foreach(_.stop()) + assert(serverInfo.isDefined, "Attempted to stop a SparkUI that was not bound to a server!") + serverInfo.get.server.stop() + logInfo("Stopped Spark Web UI at %s".format(appUIAddress)) } - private[spark] def appUIAddress = host + ":" + boundPort.getOrElse("-1") + private[spark] def appUIAddress = "http://" + host + ":" + boundPort } private[spark] object SparkUI { val DEFAULT_PORT = "4040" - val STATIC_RESOURCE_DIR = "org/apache/spark/ui" + val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static" } diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 547a194d58a5c..a487924effbff 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -19,38 +19,43 @@ package org.apache.spark.ui import scala.xml.Node -import org.apache.spark.SparkContext - /** Utility functions for generating XML pages with spark content. */ private[spark] object UIUtils { + import Page._ // Yarn has to go through a proxy so the base uri is provided and has to be on all links private[spark] val uiRoot : String = Option(System.getenv("APPLICATION_WEB_PROXY_BASE")). getOrElse("") - def prependBaseUri(resource: String = "") = uiRoot + resource + def prependBaseUri(basePath: String = "", resource: String = "") = uiRoot + basePath + resource /** Returns a spark page with correctly formatted headers */ - def headerSparkPage(content: => Seq[Node], sc: SparkContext, title: String, page: Page.Value) - : Seq[Node] = { + def headerSparkPage( + content: => Seq[Node], + basePath: String, + appName: String, + title: String, + page: Page.Value) : Seq[Node] = { val jobs = page match { - case Stages =>
  • Stages
  • - case _ =>
  • Stages
  • + case Stages => +
  • Stages
  • + case _ =>
  • Stages
  • } val storage = page match { - case Storage =>
  • Storage
  • - case _ =>
  • Storage
  • + case Storage => +
  • Storage
  • + case _ =>
  • Storage
  • } val environment = page match { case Environment => -
  • Environment
  • - case _ =>
  • Environment
  • +
  • Environment
  • + case _ =>
  • Environment
  • } val executors = page match { case Executors => -
  • Executors
  • - case _ =>
  • Executors
  • +
  • Executors
  • + case _ =>
  • Executors
  • } @@ -58,14 +63,15 @@ private[spark] object UIUtils { - + - {sc.appName} - {title} + {appName} - {title} diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala index 14333476c0e31..23e90c34d5b33 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala @@ -19,76 +19,74 @@ package org.apache.spark.ui.env import javax.servlet.http.HttpServletRequest -import scala.collection.JavaConversions._ -import scala.util.Properties import scala.xml.Node import org.eclipse.jetty.servlet.ServletContextHandler -import org.apache.spark.SparkContext +import org.apache.spark.scheduler._ +import org.apache.spark.ui._ import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.Page.Environment -import org.apache.spark.ui.UIUtils -private[spark] class EnvironmentUI(sc: SparkContext) { +private[ui] class EnvironmentUI(parent: SparkUI) { + private val appName = parent.appName + private val basePath = parent.basePath + private var _listener: Option[EnvironmentListener] = None + + lazy val listener = _listener.get + + def start() { + _listener = Some(new EnvironmentListener) + } def getHandlers = Seq[ServletContextHandler]( createServletHandler("/environment", - createServlet((request: HttpServletRequest) => envDetails(request), sc.env.securityManager)) + (request: HttpServletRequest) => render(request), parent.securityManager, basePath) ) - def envDetails(request: HttpServletRequest): Seq[Node] = { - val jvmInformation = Seq( - ("Java Version", "%s (%s)".format(Properties.javaVersion, Properties.javaVendor)), - ("Java Home", Properties.javaHome), - ("Scala Version", Properties.versionString), - ("Scala Home", Properties.scalaHome) - ).sorted - def jvmRow(kv: (String, String)) = {kv._1}{kv._2} - def jvmTable = - UIUtils.listingTable(Seq("Name", "Value"), jvmRow, jvmInformation, fixedWidth = true) - - val sparkProperties = sc.conf.getAll.sorted - - val systemProperties = System.getProperties.iterator.toSeq - val classPathProperty = systemProperties.find { case (k, v) => - k == "java.class.path" - }.getOrElse(("", "")) - val otherProperties = systemProperties.filter { case (k, v) => - k != "java.class.path" && !k.startsWith("spark.") - }.sorted - - val propertyHeaders = Seq("Name", "Value") - def propertyRow(kv: (String, String)) = {kv._1}{kv._2} - val sparkPropertyTable = - UIUtils.listingTable(propertyHeaders, propertyRow, sparkProperties, fixedWidth = true) - val otherPropertyTable = - UIUtils.listingTable(propertyHeaders, propertyRow, otherProperties, fixedWidth = true) - - val classPathEntries = classPathProperty._2 - .split(sc.conf.get("path.separator", ":")) - .filterNot(e => e.isEmpty) - .map(e => (e, "System Classpath")) - val addedJars = sc.addedJars.iterator.toSeq.map{case (path, time) => (path, "Added By User")} - val addedFiles = sc.addedFiles.iterator.toSeq.map{case (path, time) => (path, "Added By User")} - val classPath = (addedJars ++ addedFiles ++ classPathEntries).sorted - - val classPathHeaders = Seq("Resource", "Source") - def classPathRow(data: (String, String)) = {data._1}{data._2} - val classPathTable = - UIUtils.listingTable(classPathHeaders, classPathRow, classPath, fixedWidth = true) - + def render(request: HttpServletRequest): Seq[Node] = { + val runtimeInformationTable = UIUtils.listingTable( + propertyHeader, jvmRow, listener.jvmInformation, fixedWidth = true) + val sparkPropertiesTable = UIUtils.listingTable( + propertyHeader, propertyRow, listener.sparkProperties, fixedWidth = true) + val systemPropertiesTable = UIUtils.listingTable( + propertyHeader, propertyRow, listener.systemProperties, fixedWidth = true) + val classpathEntriesTable = UIUtils.listingTable( + classPathHeaders, classPathRow, listener.classpathEntries, fixedWidth = true) val content = -

    Runtime Information

    {jvmTable} -

    Spark Properties

    - {sparkPropertyTable} -

    System Properties

    - {otherPropertyTable} -

    Classpath Entries

    - {classPathTable} +

    Runtime Information

    {runtimeInformationTable} +

    Spark Properties

    {sparkPropertiesTable} +

    System Properties

    {systemPropertiesTable} +

    Classpath Entries

    {classpathEntriesTable}
    - UIUtils.headerSparkPage(content, sc, "Environment", Environment) + UIUtils.headerSparkPage(content, basePath, appName, "Environment", Environment) + } + + private def propertyHeader = Seq("Name", "Value") + private def classPathHeaders = Seq("Resource", "Source") + private def jvmRow(kv: (String, String)) = {kv._1}{kv._2} + private def propertyRow(kv: (String, String)) = {kv._1}{kv._2} + private def classPathRow(data: (String, String)) = {data._1}{data._2} +} + +/** + * A SparkListener that prepares information to be displayed on the EnvironmentUI + */ +private[ui] class EnvironmentListener extends SparkListener { + var jvmInformation = Seq[(String, String)]() + var sparkProperties = Seq[(String, String)]() + var systemProperties = Seq[(String, String)]() + var classpathEntries = Seq[(String, String)]() + + override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { + synchronized { + val environmentDetails = environmentUpdate.environmentDetails + jvmInformation = environmentDetails("JVM Information") + sparkProperties = environmentDetails("Spark Properties") + systemProperties = environmentDetails("System Properties") + classpathEntries = environmentDetails("Classpath Entries") + } } } diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala index 4235cfeff9fa2..031ed88a493a8 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala @@ -19,69 +19,43 @@ package org.apache.spark.ui.exec import javax.servlet.http.HttpServletRequest -import scala.collection.mutable.{HashMap, HashSet} +import scala.collection.mutable.HashMap import scala.xml.Node import org.eclipse.jetty.servlet.ServletContextHandler -import org.apache.spark.{ExceptionFailure, Logging, SparkContext} -import org.apache.spark.executor.TaskMetrics -import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd, SparkListenerTaskStart} -import org.apache.spark.scheduler.TaskInfo +import org.apache.spark.ExceptionFailure +import org.apache.spark.scheduler._ +import org.apache.spark.storage.StorageStatusListener import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.Page.Executors -import org.apache.spark.ui.UIUtils +import org.apache.spark.ui.{SparkUI, UIUtils} import org.apache.spark.util.Utils -private[spark] class ExecutorsUI(val sc: SparkContext) { - +private[ui] class ExecutorsUI(parent: SparkUI) { + private val appName = parent.appName + private val basePath = parent.basePath private var _listener: Option[ExecutorsListener] = None - def listener = _listener.get + + lazy val listener = _listener.get def start() { - _listener = Some(new ExecutorsListener) - sc.addSparkListener(listener) + _listener = Some(new ExecutorsListener(parent.storageStatusListener)) } def getHandlers = Seq[ServletContextHandler]( - createServletHandler("/executors", createServlet((request: HttpServletRequest) => render - (request), sc.env.securityManager)) + createServletHandler("/executors", + (request: HttpServletRequest) => render(request), parent.securityManager, basePath) ) def render(request: HttpServletRequest): Seq[Node] = { - val storageStatusList = sc.getExecutorStorageStatus - + val storageStatusList = listener.storageStatusList val maxMem = storageStatusList.map(_.maxMem).fold(0L)(_ + _) val memUsed = storageStatusList.map(_.memUsed()).fold(0L)(_ + _) val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)).fold(0L)(_ + _) - - val execHead = Seq("Executor ID", "Address", "RDD blocks", "Memory used", "Disk used", - "Active tasks", "Failed tasks", "Complete tasks", "Total tasks", "Task Time", "Shuffle Read", - "Shuffle Write") - - def execRow(kv: Seq[String]) = { - - {kv(0)} - {kv(1)} - {kv(2)} - - {Utils.bytesToString(kv(3).toLong)} / {Utils.bytesToString(kv(4).toLong)} - - - {Utils.bytesToString(kv(5).toLong)} - - {kv(6)} - {kv(7)} - {kv(8)} - {kv(9)} - {Utils.msDurationToString(kv(10).toLong)} - {Utils.bytesToString(kv(11).toLong)} - {Utils.bytesToString(kv(12).toLong)} - - } - val execInfo = for (statusId <- 0 until storageStatusList.size) yield getExecInfo(statusId) - val execTable = UIUtils.listingTable(execHead, execRow, execInfo) + val execInfoSorted = execInfo.sortBy(_.getOrElse("Executor ID", "")) + val execTable = UIUtils.listingTable(execHeader, execRow, execInfoSorted) val content =
    @@ -100,18 +74,61 @@ private[spark] class ExecutorsUI(val sc: SparkContext) {
    ; - UIUtils.headerSparkPage(content, sc, "Executors (" + execInfo.size + ")", Executors) + UIUtils.headerSparkPage( + content, basePath, appName, "Executors (" + execInfo.size + ")", Executors) + } + + /** Header fields for the executors table */ + private def execHeader = Seq( + "Executor ID", + "Address", + "RDD Blocks", + "Memory Used", + "Disk Used", + "Active Tasks", + "Failed Tasks", + "Complete Tasks", + "Total Tasks", + "Task Time", + "Shuffle Read", + "Shuffle Write") + + /** Render an HTML row representing an executor */ + private def execRow(values: Map[String, String]): Seq[Node] = { + val maximumMemory = values("Maximum Memory") + val memoryUsed = values("Memory Used") + val diskUsed = values("Disk Used") + + {values("Executor ID")} + {values("Address")} + {values("RDD Blocks")} + + {Utils.bytesToString(memoryUsed.toLong)} / + {Utils.bytesToString(maximumMemory.toLong)} + + + {Utils.bytesToString(diskUsed.toLong)} + + {values("Active Tasks")} + {values("Failed Tasks")} + {values("Complete Tasks")} + {values("Total Tasks")} + {Utils.msDurationToString(values("Task Time").toLong)} + {Utils.bytesToString(values("Shuffle Read").toLong)} + {Utils.bytesToString(values("Shuffle Write").toLong)} + } - def getExecInfo(statusId: Int): Seq[String] = { - val status = sc.getExecutorStorageStatus(statusId) + /** Represent an executor's info as a map given a storage status index */ + private def getExecInfo(statusId: Int): Map[String, String] = { + val status = listener.storageStatusList(statusId) val execId = status.blockManagerId.executorId val hostPort = status.blockManagerId.hostPort - val rddBlocks = status.blocks.size.toString - val memUsed = status.memUsed().toString - val maxMem = status.maxMem.toString - val diskUsed = status.diskUsed().toString - val activeTasks = listener.executorToTasksActive.getOrElse(execId, HashSet.empty[Long]).size + val rddBlocks = status.blocks.size + val memUsed = status.memUsed() + val maxMem = status.maxMem + val diskUsed = status.diskUsed() + val activeTasks = listener.executorToTasksActive.getOrElse(execId, 0) val failedTasks = listener.executorToTasksFailed.getOrElse(execId, 0) val completedTasks = listener.executorToTasksComplete.getOrElse(execId, 0) val totalTasks = activeTasks + failedTasks + completedTasks @@ -119,64 +136,77 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { val totalShuffleRead = listener.executorToShuffleRead.getOrElse(execId, 0) val totalShuffleWrite = listener.executorToShuffleWrite.getOrElse(execId, 0) - Seq( + // Also include fields not in the header + val execFields = execHeader ++ Seq("Maximum Memory") + + val execValues = Seq( execId, hostPort, rddBlocks, memUsed, - maxMem, diskUsed, - activeTasks.toString, - failedTasks.toString, - completedTasks.toString, - totalTasks.toString, - totalDuration.toString, - totalShuffleRead.toString, - totalShuffleWrite.toString - ) + activeTasks, + failedTasks, + completedTasks, + totalTasks, + totalDuration, + totalShuffleRead, + totalShuffleWrite, + maxMem + ).map(_.toString) + + execFields.zip(execValues).toMap } +} - private[spark] class ExecutorsListener extends SparkListener with Logging { - val executorToTasksActive = HashMap[String, HashSet[TaskInfo]]() - val executorToTasksComplete = HashMap[String, Int]() - val executorToTasksFailed = HashMap[String, Int]() - val executorToDuration = HashMap[String, Long]() - val executorToShuffleRead = HashMap[String, Long]() - val executorToShuffleWrite = HashMap[String, Long]() - - override def onTaskStart(taskStart: SparkListenerTaskStart) { - val eid = taskStart.taskInfo.executorId - val activeTasks = executorToTasksActive.getOrElseUpdate(eid, new HashSet[TaskInfo]()) - activeTasks += taskStart.taskInfo - } +/** + * A SparkListener that prepares information to be displayed on the ExecutorsUI + */ +private[ui] class ExecutorsListener(storageStatusListener: StorageStatusListener) + extends SparkListener { - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { - val eid = taskEnd.taskInfo.executorId - val activeTasks = executorToTasksActive.getOrElseUpdate(eid, new HashSet[TaskInfo]()) - val newDuration = executorToDuration.getOrElse(eid, 0L) + taskEnd.taskInfo.duration - executorToDuration.put(eid, newDuration) - - activeTasks -= taskEnd.taskInfo - val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = - taskEnd.reason match { - case e: ExceptionFailure => - executorToTasksFailed(eid) = executorToTasksFailed.getOrElse(eid, 0) + 1 - (Some(e), e.metrics) - case _ => - executorToTasksComplete(eid) = executorToTasksComplete.getOrElse(eid, 0) + 1 - (None, Option(taskEnd.taskMetrics)) - } + val executorToTasksActive = HashMap[String, Int]() + val executorToTasksComplete = HashMap[String, Int]() + val executorToTasksFailed = HashMap[String, Int]() + val executorToDuration = HashMap[String, Long]() + val executorToShuffleRead = HashMap[String, Long]() + val executorToShuffleWrite = HashMap[String, Long]() + + def storageStatusList = storageStatusListener.storageStatusList + + override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { + val eid = formatExecutorId(taskStart.taskInfo.executorId) + executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 0) + 1 + } - // update shuffle read/write - if (null != taskEnd.taskMetrics) { - taskEnd.taskMetrics.shuffleReadMetrics.foreach(shuffleRead => - executorToShuffleRead.put(eid, executorToShuffleRead.getOrElse(eid, 0L) + - shuffleRead.remoteBytesRead)) + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { + val info = taskEnd.taskInfo + if (info != null) { + val eid = formatExecutorId(info.executorId) + executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 1) - 1 + executorToDuration(eid) = executorToDuration.getOrElse(eid, 0L) + info.duration + taskEnd.reason match { + case e: ExceptionFailure => + executorToTasksFailed(eid) = executorToTasksFailed.getOrElse(eid, 0) + 1 + case _ => + executorToTasksComplete(eid) = executorToTasksComplete.getOrElse(eid, 0) + 1 + } - taskEnd.taskMetrics.shuffleWriteMetrics.foreach(shuffleWrite => - executorToShuffleWrite.put(eid, executorToShuffleWrite.getOrElse(eid, 0L) + - shuffleWrite.shuffleBytesWritten)) + // Update shuffle read/write + val metrics = taskEnd.taskMetrics + if (metrics != null) { + metrics.shuffleReadMetrics.foreach { shuffleRead => + executorToShuffleRead(eid) = + executorToShuffleRead.getOrElse(eid, 0L) + shuffleRead.remoteBytesRead + } + metrics.shuffleWriteMetrics.foreach { shuffleWrite => + executorToShuffleWrite(eid) = + executorToShuffleWrite.getOrElse(eid, 0L) + shuffleWrite.shuffleBytesWritten + } } } } + + // This addresses executor ID inconsistencies in the local mode + private def formatExecutorId(execId: String) = storageStatusListener.formatExecutorId(execId) } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala index 64e22a30b48f9..1dfe1d4f1fa11 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala @@ -18,7 +18,7 @@ package org.apache.spark.ui.jobs /** class for reporting aggregated metrics for each executors in stageUI */ -private[spark] class ExecutorSummary { +private[ui] class ExecutorSummary { var taskTime : Long = 0 var failedTasks : Int = 0 var succeededTasks : Int = 0 diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index d012ba4dbb3db..73861ae6746da 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -20,17 +20,13 @@ package org.apache.spark.ui.jobs import scala.collection.mutable import scala.xml.Node -import org.apache.spark.scheduler.SchedulingMode import org.apache.spark.util.Utils /** Page showing executor summary */ -private[spark] class ExecutorTable(val parent: JobProgressUI, val stageId: Int) { +private[ui] class ExecutorTable(stageId: Int, parent: JobProgressUI) { + private lazy val listener = parent.listener - val listener = parent.listener - val dateFmt = parent.dateFmt - val isFairScheduler = listener.sc.getSchedulingMode == SchedulingMode.FAIR - - def toNodeSeq(): Seq[Node] = { + def toNodeSeq: Seq[Node] = { listener.synchronized { executorTable() } @@ -58,11 +54,9 @@ private[spark] class ExecutorTable(val parent: JobProgressUI, val stageId: Int) } private def createExecutorTable() : Seq[Node] = { - // make a executor-id -> address map + // Make an executor-id -> address map val executorIdToAddress = mutable.HashMap[String, String]() - val storageStatusList = parent.sc.getExecutorStorageStatus - for (statusId <- 0 until storageStatusList.size) { - val blockManagerId = parent.sc.getExecutorStorageStatus(statusId).blockManagerId + listener.blockManagerIds.foreach { blockManagerId => val address = blockManagerId.hostPort val executorId = blockManagerId.executorId executorIdToAddress.put(executorId, address) @@ -70,25 +64,23 @@ private[spark] class ExecutorTable(val parent: JobProgressUI, val stageId: Int) val executorIdToSummary = listener.stageIdToExecutorSummaries.get(stageId) executorIdToSummary match { - case Some(x) => { - x.toSeq.sortBy(_._1).map{ - case (k,v) => { - - {k} - {executorIdToAddress.getOrElse(k, "CANNOT FIND ADDRESS")} - {parent.formatDuration(v.taskTime)} - {v.failedTasks + v.succeededTasks} - {v.failedTasks} - {v.succeededTasks} - {Utils.bytesToString(v.shuffleRead)} - {Utils.bytesToString(v.shuffleWrite)} - {Utils.bytesToString(v.memoryBytesSpilled)} - {Utils.bytesToString(v.diskBytesSpilled)} - - } + case Some(x) => + x.toSeq.sortBy(_._1).map { case (k, v) => { + + {k} + {executorIdToAddress.getOrElse(k, "CANNOT FIND ADDRESS")} + {parent.formatDuration(v.taskTime)} + {v.failedTasks + v.succeededTasks} + {v.failedTasks} + {v.succeededTasks} + {Utils.bytesToString(v.shuffleRead)} + {Utils.bytesToString(v.shuffleWrite)} + {Utils.bytesToString(v.memoryBytesSpilled)} + {Utils.bytesToString(v.diskBytesSpilled)} + } } - case _ => { Seq[Node]() } + case _ => Seq[Node]() } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala index 81713edcf5db2..f3c93d4214ad0 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala @@ -19,72 +19,80 @@ package org.apache.spark.ui.jobs import javax.servlet.http.HttpServletRequest -import scala.xml.{NodeSeq, Node} +import scala.xml.{Node, NodeSeq} -import org.apache.spark.scheduler.SchedulingMode +import org.apache.spark.scheduler.Schedulable import org.apache.spark.ui.Page._ -import org.apache.spark.ui.UIUtils._ +import org.apache.spark.ui.UIUtils /** Page showing list of all ongoing and recently finished stages and pools*/ -private[spark] class IndexPage(parent: JobProgressUI) { - def listener = parent.listener +private[ui] class IndexPage(parent: JobProgressUI) { + private val appName = parent.appName + private val basePath = parent.basePath + private val live = parent.live + private val sc = parent.sc + private lazy val listener = parent.listener + private lazy val isFairScheduler = parent.isFairScheduler def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { - val activeStages = listener.activeStages.toSeq + val activeStages = listener.activeStages.values.toSeq val completedStages = listener.completedStages.reverse.toSeq val failedStages = listener.failedStages.reverse.toSeq val now = System.currentTimeMillis() - var activeTime = 0L - for (tasks <- listener.stageIdToTasksActive.values; t <- tasks) { - activeTime += t.timeRunning(now) - } - val activeStagesTable = new StageTable(activeStages.sortBy(_.submissionTime).reverse, parent) - val completedStagesTable = new StageTable(completedStages.sortBy(_.submissionTime).reverse, - parent) + val completedStagesTable = + new StageTable(completedStages.sortBy(_.submissionTime).reverse, parent) val failedStagesTable = new StageTable(failedStages.sortBy(_.submissionTime).reverse, parent) - val pools = listener.sc.getAllPools - val poolTable = new PoolTable(pools, listener) + // For now, pool information is only accessible in live UIs + val pools = if (live) sc.getAllPools else Seq[Schedulable]() + val poolTable = new PoolTable(pools, parent) + val summary: NodeSeq = -
    -
      -
    • - Total Duration: - {parent.formatDuration(now - listener.sc.startTime)} -
    • -
    • Scheduling Mode: {parent.sc.getSchedulingMode}
    • -
    • - Active Stages: - {activeStages.size} -
    • -
    • - Completed Stages: - {completedStages.size} -
    • -
    • +
      +
        + {if (live) { + // Total duration is not meaningful unless the UI is live +
      • + Total Duration: + {parent.formatDuration(now - sc.startTime)} +
      • + }} +
      • + Scheduling Mode: + {listener.schedulingMode.map(_.toString).getOrElse("Unknown")} +
      • +
      • + Active Stages: + {activeStages.size} +
      • +
      • + Completed Stages: + {completedStages.size} +
      • +
      • Failed Stages: - {failedStages.size} -
      • -
      -
      + {failedStages.size} +
    • +
    +
    val content = summary ++ - {if (listener.sc.getSchedulingMode == SchedulingMode.FAIR) { -

    {pools.size} Fair Scheduler Pools

    ++ poolTable.toNodeSeq + {if (live && isFairScheduler) { +

    {pools.size} Fair Scheduler Pools

    ++ poolTable.toNodeSeq } else { - Seq() + Seq[Node]() }} ++

    Active Stages ({activeStages.size})

    ++ - activeStagesTable.toNodeSeq++ + activeStagesTable.toNodeSeq ++

    Completed Stages ({completedStages.size})

    ++ - completedStagesTable.toNodeSeq++ + completedStagesTable.toNodeSeq ++

    Failed Stages ({failedStages.size})

    ++ failedStagesTable.toNodeSeq - headerSparkPage(content, parent.sc, "Spark Stages", Stages) + UIUtils.headerSparkPage(content, basePath, appName, "Spark Stages", Stages) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 07a08f5277d19..d10aa12b9ebca 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -17,29 +17,29 @@ package org.apache.spark.ui.jobs -import scala.collection.mutable.{ListBuffer, HashMap, HashSet} +import scala.collection.mutable.{HashMap, ListBuffer} -import org.apache.spark.{ExceptionFailure, SparkContext, Success} +import org.apache.spark.{ExceptionFailure, SparkConf, SparkContext, Success} import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler._ +import org.apache.spark.scheduler.SchedulingMode.SchedulingMode +import org.apache.spark.storage.BlockManagerId /** * Tracks task-level information to be displayed in the UI. * * All access to the data structures in this class must be synchronized on the - * class, since the UI thread and the DAGScheduler event loop may otherwise - * be reading/updating the internal data structures concurrently. + * class, since the UI thread and the EventBus loop may otherwise be reading and + * updating the internal data structures concurrently. */ -private[spark] class JobProgressListener(val sc: SparkContext) extends SparkListener { - // How many stages to remember - val RETAINED_STAGES = sc.conf.getInt("spark.ui.retainedStages", 1000) - val DEFAULT_POOL_NAME = "default" +private[ui] class JobProgressListener(conf: SparkConf) extends SparkListener { - val stageIdToPool = new HashMap[Int, String]() - val stageIdToDescription = new HashMap[Int, String]() - val poolToActiveStages = new HashMap[String, HashSet[StageInfo]]() + import JobProgressListener._ - val activeStages = HashSet[StageInfo]() + // How many stages to remember + val retainedStages = conf.getInt("spark.ui.retainedStages", DEFAULT_RETAINED_STAGES) + + val activeStages = HashMap[Int, StageInfo]() val completedStages = ListBuffer[StageInfo]() val failedStages = ListBuffer[StageInfo]() @@ -53,29 +53,37 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList val stageIdToShuffleWrite = HashMap[Int, Long]() val stageIdToMemoryBytesSpilled = HashMap[Int, Long]() val stageIdToDiskBytesSpilled = HashMap[Int, Long]() - val stageIdToTasksActive = HashMap[Int, HashSet[TaskInfo]]() + val stageIdToTasksActive = HashMap[Int, HashMap[Long, TaskInfo]]() val stageIdToTasksComplete = HashMap[Int, Int]() val stageIdToTasksFailed = HashMap[Int, Int]() - val stageIdToTaskInfos = - HashMap[Int, HashSet[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]]() + val stageIdToTaskData = HashMap[Int, HashMap[Long, TaskUIData]]() val stageIdToExecutorSummaries = HashMap[Int, HashMap[String, ExecutorSummary]]() + val stageIdToPool = HashMap[Int, String]() + val stageIdToDescription = HashMap[Int, String]() + val poolToActiveStages = HashMap[String, HashMap[Int, StageInfo]]() + + val executorIdToBlockManagerId = HashMap[String, BlockManagerId]() - override def onJobStart(jobStart: SparkListenerJobStart) {} + var schedulingMode: Option[SchedulingMode] = None + + def blockManagerIds = executorIdToBlockManagerId.values.toSeq override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = synchronized { - val stage = stageCompleted.stage - poolToActiveStages(stageIdToPool(stage.stageId)) -= stage - activeStages -= stage + val stage = stageCompleted.stageInfo + val stageId = stage.stageId + // Remove by stageId, rather than by StageInfo, in case the StageInfo is from storage + poolToActiveStages(stageIdToPool(stageId)).remove(stageId) + activeStages.remove(stageId) completedStages += stage trimIfNecessary(completedStages) } /** If stages is too large, remove and garbage collect old stages */ - def trimIfNecessary(stages: ListBuffer[StageInfo]) = synchronized { - if (stages.size > RETAINED_STAGES) { - val toRemove = RETAINED_STAGES / 10 + private def trimIfNecessary(stages: ListBuffer[StageInfo]) = synchronized { + if (stages.size > retainedStages) { + val toRemove = retainedStages / 10 stages.takeRight(toRemove).foreach( s => { - stageIdToTaskInfos.remove(s.stageId) + stageIdToTaskData.remove(s.stageId) stageIdToTime.remove(s.stageId) stageIdToShuffleRead.remove(s.stageId) stageIdToShuffleWrite.remove(s.stageId) @@ -93,8 +101,8 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList /** For FIFO, all stages are contained by "default" pool but "default" pool here is meaningless */ override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = synchronized { - val stage = stageSubmitted.stage - activeStages += stage + val stage = stageSubmitted.stageInfo + activeStages(stage.stageId) = stage val poolName = Option(stageSubmitted.properties).map { p => p.getProperty("spark.scheduler.pool", DEFAULT_POOL_NAME) @@ -106,121 +114,154 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList } description.map(d => stageIdToDescription(stage.stageId) = d) - val stages = poolToActiveStages.getOrElseUpdate(poolName, new HashSet[StageInfo]()) - stages += stage + val stages = poolToActiveStages.getOrElseUpdate(poolName, new HashMap[Int, StageInfo]()) + stages(stage.stageId) = stage } override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { - val sid = taskStart.task.stageId - val tasksActive = stageIdToTasksActive.getOrElseUpdate(sid, new HashSet[TaskInfo]()) - tasksActive += taskStart.taskInfo - val taskList = stageIdToTaskInfos.getOrElse( - sid, HashSet[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) - taskList += ((taskStart.taskInfo, None, None)) - stageIdToTaskInfos(sid) = taskList + val sid = taskStart.stageId + val taskInfo = taskStart.taskInfo + if (taskInfo != null) { + val tasksActive = stageIdToTasksActive.getOrElseUpdate(sid, new HashMap[Long, TaskInfo]()) + tasksActive(taskInfo.taskId) = taskInfo + val taskMap = stageIdToTaskData.getOrElse(sid, HashMap[Long, TaskUIData]()) + taskMap(taskInfo.taskId) = new TaskUIData(taskInfo) + stageIdToTaskData(sid) = taskMap + } } - override def onTaskGettingResult(taskGettingResult: SparkListenerTaskGettingResult) - = synchronized { + override def onTaskGettingResult(taskGettingResult: SparkListenerTaskGettingResult) { // Do nothing: because we don't do a deep copy of the TaskInfo, the TaskInfo in // stageToTaskInfos already has the updated status. } override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { - val sid = taskEnd.task.stageId - - // create executor summary map if necessary - val executorSummaryMap = stageIdToExecutorSummaries.getOrElseUpdate(key = sid, - op = new HashMap[String, ExecutorSummary]()) - executorSummaryMap.getOrElseUpdate(key = taskEnd.taskInfo.executorId, - op = new ExecutorSummary()) - - val executorSummary = executorSummaryMap.get(taskEnd.taskInfo.executorId) - executorSummary match { - case Some(y) => { - // first update failed-task, succeed-task + val sid = taskEnd.stageId + val info = taskEnd.taskInfo + + if (info != null) { + // create executor summary map if necessary + val executorSummaryMap = stageIdToExecutorSummaries.getOrElseUpdate(key = sid, + op = new HashMap[String, ExecutorSummary]()) + executorSummaryMap.getOrElseUpdate(key = info.executorId, op = new ExecutorSummary) + + val executorSummary = executorSummaryMap.get(info.executorId) + executorSummary match { + case Some(y) => { + // first update failed-task, succeed-task + taskEnd.reason match { + case Success => + y.succeededTasks += 1 + case _ => + y.failedTasks += 1 + } + + // update duration + y.taskTime += info.duration + + val metrics = taskEnd.taskMetrics + if (metrics != null) { + metrics.shuffleReadMetrics.foreach { y.shuffleRead += _.remoteBytesRead } + metrics.shuffleWriteMetrics.foreach { y.shuffleWrite += _.shuffleBytesWritten } + y.memoryBytesSpilled += metrics.memoryBytesSpilled + y.diskBytesSpilled += metrics.diskBytesSpilled + } + } + case _ => {} + } + + val tasksActive = stageIdToTasksActive.getOrElseUpdate(sid, new HashMap[Long, TaskInfo]()) + // Remove by taskId, rather than by TaskInfo, in case the TaskInfo is from storage + tasksActive.remove(info.taskId) + + val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { - case Success => - y.succeededTasks += 1 + case e: ExceptionFailure => + stageIdToTasksFailed(sid) = stageIdToTasksFailed.getOrElse(sid, 0) + 1 + (Some(e), e.metrics) case _ => - y.failedTasks += 1 + stageIdToTasksComplete(sid) = stageIdToTasksComplete.getOrElse(sid, 0) + 1 + (None, Option(taskEnd.taskMetrics)) } - // update duration - y.taskTime += taskEnd.taskInfo.duration + stageIdToTime.getOrElseUpdate(sid, 0L) + val time = metrics.map(_.executorRunTime).getOrElse(0L) + stageIdToTime(sid) += time + totalTime += time - Option(taskEnd.taskMetrics).foreach { taskMetrics => - taskMetrics.shuffleReadMetrics.foreach { y.shuffleRead += _.remoteBytesRead } - taskMetrics.shuffleWriteMetrics.foreach { y.shuffleWrite += _.shuffleBytesWritten } - y.memoryBytesSpilled += taskMetrics.memoryBytesSpilled - y.diskBytesSpilled += taskMetrics.diskBytesSpilled - } - } - case _ => {} - } + stageIdToShuffleRead.getOrElseUpdate(sid, 0L) + val shuffleRead = metrics.flatMap(_.shuffleReadMetrics).map(_.remoteBytesRead).getOrElse(0L) + stageIdToShuffleRead(sid) += shuffleRead + totalShuffleRead += shuffleRead - val tasksActive = stageIdToTasksActive.getOrElseUpdate(sid, new HashSet[TaskInfo]()) - tasksActive -= taskEnd.taskInfo - - val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = - taskEnd.reason match { - case e: ExceptionFailure => - stageIdToTasksFailed(sid) = stageIdToTasksFailed.getOrElse(sid, 0) + 1 - (Some(e), e.metrics) - case _ => - stageIdToTasksComplete(sid) = stageIdToTasksComplete.getOrElse(sid, 0) + 1 - (None, Option(taskEnd.taskMetrics)) - } + stageIdToShuffleWrite.getOrElseUpdate(sid, 0L) + val shuffleWrite = + metrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleBytesWritten).getOrElse(0L) + stageIdToShuffleWrite(sid) += shuffleWrite + totalShuffleWrite += shuffleWrite - stageIdToTime.getOrElseUpdate(sid, 0L) - val time = metrics.map(m => m.executorRunTime).getOrElse(0) - stageIdToTime(sid) += time - totalTime += time - - stageIdToShuffleRead.getOrElseUpdate(sid, 0L) - val shuffleRead = metrics.flatMap(m => m.shuffleReadMetrics).map(s => - s.remoteBytesRead).getOrElse(0L) - stageIdToShuffleRead(sid) += shuffleRead - totalShuffleRead += shuffleRead - - stageIdToShuffleWrite.getOrElseUpdate(sid, 0L) - val shuffleWrite = metrics.flatMap(m => m.shuffleWriteMetrics).map(s => - s.shuffleBytesWritten).getOrElse(0L) - stageIdToShuffleWrite(sid) += shuffleWrite - totalShuffleWrite += shuffleWrite - - stageIdToMemoryBytesSpilled.getOrElseUpdate(sid, 0L) - val memoryBytesSpilled = metrics.map(m => m.memoryBytesSpilled).getOrElse(0L) - stageIdToMemoryBytesSpilled(sid) += memoryBytesSpilled - - stageIdToDiskBytesSpilled.getOrElseUpdate(sid, 0L) - val diskBytesSpilled = metrics.map(m => m.diskBytesSpilled).getOrElse(0L) - stageIdToDiskBytesSpilled(sid) += diskBytesSpilled - - val taskList = stageIdToTaskInfos.getOrElse( - sid, HashSet[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) - taskList -= ((taskEnd.taskInfo, None, None)) - taskList += ((taskEnd.taskInfo, metrics, failureInfo)) - stageIdToTaskInfos(sid) = taskList + stageIdToMemoryBytesSpilled.getOrElseUpdate(sid, 0L) + val memoryBytesSpilled = metrics.map(_.memoryBytesSpilled).getOrElse(0L) + stageIdToMemoryBytesSpilled(sid) += memoryBytesSpilled + + stageIdToDiskBytesSpilled.getOrElseUpdate(sid, 0L) + val diskBytesSpilled = metrics.map(_.diskBytesSpilled).getOrElse(0L) + stageIdToDiskBytesSpilled(sid) += diskBytesSpilled + + val taskMap = stageIdToTaskData.getOrElse(sid, HashMap[Long, TaskUIData]()) + taskMap(info.taskId) = new TaskUIData(info, metrics, failureInfo) + stageIdToTaskData(sid) = taskMap + } } override def onJobEnd(jobEnd: SparkListenerJobEnd) = synchronized { - jobEnd match { - case end: SparkListenerJobEnd => - end.jobResult match { - case JobFailed(ex, Some(stage)) => - /* If two jobs share a stage we could get this failure message twice. So we first - * check whether we've already retired this stage. */ - val stageInfo = activeStages.filter(s => s.stageId == stage.id).headOption - stageInfo.foreach {s => - activeStages -= s - poolToActiveStages(stageIdToPool(stage.id)) -= s - failedStages += s - trimIfNecessary(failedStages) - } - case _ => + jobEnd.jobResult match { + case JobFailed(_, stageId) => + activeStages.get(stageId).foreach { s => + // Remove by stageId, rather than by StageInfo, in case the StageInfo is from storage + activeStages.remove(s.stageId) + poolToActiveStages(stageIdToPool(stageId)).remove(s.stageId) + failedStages += s + trimIfNecessary(failedStages) } case _ => } } + + override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { + synchronized { + val schedulingModeName = + environmentUpdate.environmentDetails("Spark Properties").toMap.get("spark.scheduler.mode") + schedulingMode = schedulingModeName match { + case Some(name) => Some(SchedulingMode.withName(name)) + case None => None + } + } + } + + override def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded) { + synchronized { + val blockManagerId = blockManagerAdded.blockManagerId + val executorId = blockManagerId.executorId + executorIdToBlockManagerId(executorId) = blockManagerId + } + } + + override def onBlockManagerRemoved(blockManagerRemoved: SparkListenerBlockManagerRemoved) { + synchronized { + val executorId = blockManagerRemoved.blockManagerId.executorId + executorIdToBlockManagerId.remove(executorId) + } + } + +} + +private[ui] case class TaskUIData( + taskInfo: TaskInfo, + taskMetrics: Option[TaskMetrics] = None, + exception: Option[ExceptionFailure] = None) + +private object JobProgressListener { + val DEFAULT_POOL_NAME = "default" + val DEFAULT_RETAINED_STAGES = 1000 } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala index 2d95d47e154cd..ee4e9c69c1bd1 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala @@ -20,41 +20,43 @@ package org.apache.spark.ui.jobs import java.text.SimpleDateFormat import javax.servlet.http.HttpServletRequest -import scala.Seq - -import org.eclipse.jetty.server.Handler import org.eclipse.jetty.servlet.ServletContextHandler -import org.apache.spark.SparkContext +import org.apache.spark.SparkConf +import org.apache.spark.scheduler.SchedulingMode import org.apache.spark.ui.JettyUtils._ +import org.apache.spark.ui.SparkUI import org.apache.spark.util.Utils /** Web UI showing progress status of all jobs in the given SparkContext. */ -private[spark] class JobProgressUI(val sc: SparkContext) { - private var _listener: Option[JobProgressListener] = None - def listener = _listener.get +private[ui] class JobProgressUI(parent: SparkUI) { + val appName = parent.appName + val basePath = parent.basePath val dateFmt = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + val live = parent.live + val sc = parent.sc + + lazy val listener = _listener.get + lazy val isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) private val indexPage = new IndexPage(this) private val stagePage = new StagePage(this) private val poolPage = new PoolPage(this) + private var _listener: Option[JobProgressListener] = None def start() { - _listener = Some(new JobProgressListener(sc)) - sc.addSparkListener(listener) + val conf = if (live) sc.conf else new SparkConf + _listener = Some(new JobProgressListener(conf)) } def formatDuration(ms: Long) = Utils.msDurationToString(ms) def getHandlers = Seq[ServletContextHandler]( createServletHandler("/stages/stage", - createServlet((request: HttpServletRequest) => stagePage.render(request), - sc.env.securityManager)), + (request: HttpServletRequest) => stagePage.render(request), parent.securityManager, basePath), createServletHandler("/stages/pool", - createServlet((request: HttpServletRequest) => poolPage.render(request), - sc.env.securityManager)), + (request: HttpServletRequest) => poolPage.render(request), parent.securityManager, basePath), createServletHandler("/stages", - createServlet((request: HttpServletRequest) => indexPage.render(request), - sc.env.securityManager)) + (request: HttpServletRequest) => indexPage.render(request), parent.securityManager, basePath) ) } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index eb7518a020840..bd33182b70059 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -21,27 +21,38 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node +import org.apache.spark.scheduler.{Schedulable, StageInfo} import org.apache.spark.ui.Page._ -import org.apache.spark.ui.UIUtils._ +import org.apache.spark.ui.UIUtils /** Page showing specific pool details */ -private[spark] class PoolPage(parent: JobProgressUI) { - def listener = parent.listener +private[ui] class PoolPage(parent: JobProgressUI) { + private val appName = parent.appName + private val basePath = parent.basePath + private val live = parent.live + private val sc = parent.sc + private lazy val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { val poolName = request.getParameter("poolname") val poolToActiveStages = listener.poolToActiveStages - val activeStages = poolToActiveStages.get(poolName).toSeq.flatten + val activeStages = poolToActiveStages.get(poolName) match { + case Some(s) => s.values.toSeq + case None => Seq[StageInfo]() + } val activeStagesTable = new StageTable(activeStages.sortBy(_.submissionTime).reverse, parent) - val pool = listener.sc.getPoolForName(poolName).get - val poolTable = new PoolTable(Seq(pool), listener) + // For now, pool information is only accessible in live UIs + val pools = if (live) Seq(sc.getPoolForName(poolName).get) else Seq[Schedulable]() + val poolTable = new PoolTable(pools, parent) - val content =

    Summary

    ++ poolTable.toNodeSeq() ++ -

    {activeStages.size} Active Stages

    ++ activeStagesTable.toNodeSeq() + val content = +

    Summary

    ++ poolTable.toNodeSeq ++ +

    {activeStages.size} Active Stages

    ++ activeStagesTable.toNodeSeq - headerSparkPage(content, parent.sc, "Fair Scheduler Pool: " + poolName, Stages) + UIUtils.headerSparkPage( + content, basePath, appName, "Fair Scheduler Pool: " + poolName, Stages) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala index 22bc97ada18be..c5c8d8668740b 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala @@ -18,26 +18,26 @@ package org.apache.spark.ui.jobs import scala.collection.mutable.HashMap -import scala.collection.mutable.HashSet import scala.xml.Node import org.apache.spark.scheduler.{Schedulable, StageInfo} import org.apache.spark.ui.UIUtils /** Table showing list of pools */ -private[spark] class PoolTable(pools: Seq[Schedulable], listener: JobProgressListener) { +private[ui] class PoolTable(pools: Seq[Schedulable], parent: JobProgressUI) { + private val basePath = parent.basePath + private val poolToActiveStages = listener.poolToActiveStages + private lazy val listener = parent.listener - var poolToActiveStages: HashMap[String, HashSet[StageInfo]] = listener.poolToActiveStages - - def toNodeSeq(): Seq[Node] = { + def toNodeSeq: Seq[Node] = { listener.synchronized { poolTable(poolRow, pools) } } - private def poolTable(makeRow: (Schedulable, HashMap[String, HashSet[StageInfo]]) => Seq[Node], - rows: Seq[Schedulable] - ): Seq[Node] = { + private def poolTable( + makeRow: (Schedulable, HashMap[String, HashMap[Int, StageInfo]]) => Seq[Node], + rows: Seq[Schedulable]): Seq[Node] = { @@ -53,15 +53,18 @@ private[spark] class PoolTable(pools: Seq[Schedulable], listener: JobProgressLis
    Pool Name
    } - private def poolRow(p: Schedulable, poolToActiveStages: HashMap[String, HashSet[StageInfo]]) - : Seq[Node] = { + private def poolRow( + p: Schedulable, + poolToActiveStages: HashMap[String, HashMap[Int, StageInfo]]): Seq[Node] = { val activeStages = poolToActiveStages.get(p.name) match { case Some(stages) => stages.size case None => 0 } - {p.name} + + {p.name} + {p.minShare} {p.weight} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index ddc687a45a095..da7f20233063e 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -22,47 +22,47 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.ExceptionFailure -import org.apache.spark.executor.TaskMetrics -import org.apache.spark.scheduler.TaskInfo -import org.apache.spark.ui.UIUtils._ import org.apache.spark.ui.Page._ +import org.apache.spark.ui.UIUtils import org.apache.spark.util.{Utils, Distribution} /** Page showing statistics and task list for a given stage */ -private[spark] class StagePage(parent: JobProgressUI) { - def listener = parent.listener - val dateFmt = parent.dateFmt +private[ui] class StagePage(parent: JobProgressUI) { + private val appName = parent.appName + private val basePath = parent.basePath + private val dateFmt = parent.dateFmt + private lazy val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { val stageId = request.getParameter("id").toInt - val now = System.currentTimeMillis() - if (!listener.stageIdToTaskInfos.contains(stageId)) { + if (!listener.stageIdToTaskData.contains(stageId)) { val content =

    Summary Metrics

    No tasks have started yet

    Tasks

    No tasks have started yet
    - return headerSparkPage(content, parent.sc, "Details for Stage %s".format(stageId), Stages) + return UIUtils.headerSparkPage( + content, basePath, appName, "Details for Stage %s".format(stageId), Stages) } - val tasks = listener.stageIdToTaskInfos(stageId).toSeq.sortBy(_._1.launchTime) + val tasks = listener.stageIdToTaskData(stageId).values.toSeq.sortBy(_.taskInfo.launchTime) - val numCompleted = tasks.count(_._1.finished) + val numCompleted = tasks.count(_.taskInfo.finished) val shuffleReadBytes = listener.stageIdToShuffleRead.getOrElse(stageId, 0L) val hasShuffleRead = shuffleReadBytes > 0 val shuffleWriteBytes = listener.stageIdToShuffleWrite.getOrElse(stageId, 0L) val hasShuffleWrite = shuffleWriteBytes > 0 val memoryBytesSpilled = listener.stageIdToMemoryBytesSpilled.getOrElse(stageId, 0L) val diskBytesSpilled = listener.stageIdToDiskBytesSpilled.getOrElse(stageId, 0L) - val hasBytesSpilled = (memoryBytesSpilled > 0 && diskBytesSpilled > 0) + val hasBytesSpilled = memoryBytesSpilled > 0 && diskBytesSpilled > 0 var activeTime = 0L - listener.stageIdToTasksActive(stageId).foreach(activeTime += _.timeRunning(now)) + val now = System.currentTimeMillis() + val tasksActive = listener.stageIdToTasksActive(stageId).values + tasksActive.foreach(activeTime += _.timeRunning(now)) - val finishedTasks = listener.stageIdToTaskInfos(stageId).filter(_._1.finished) // scalastyle:off val summary =
    @@ -104,42 +104,45 @@ private[spark] class StagePage(parent: JobProgressUI) { {if (hasBytesSpilled) Seq("Shuffle Spill (Memory)", "Shuffle Spill (Disk)") else Nil} ++ Seq("Errors") - val taskTable = listingTable( + val taskTable = UIUtils.listingTable( taskHeaders, taskRow(hasShuffleRead, hasShuffleWrite, hasBytesSpilled), tasks) // Excludes tasks which failed and have incomplete metrics - val validTasks = tasks.filter(t => t._1.status == "SUCCESS" && (t._2.isDefined)) + val validTasks = tasks.filter(t => t.taskInfo.status == "SUCCESS" && t.taskMetrics.isDefined) val summaryTable: Option[Seq[Node]] = if (validTasks.size == 0) { None } else { - val serializationTimes = validTasks.map{case (info, metrics, exception) => - metrics.get.resultSerializationTime.toDouble} + val serializationTimes = validTasks.map { case TaskUIData(_, metrics, _) => + metrics.get.resultSerializationTime.toDouble + } val serializationQuantiles = "Result serialization time" +: Distribution(serializationTimes). get.getQuantiles().map(ms => parent.formatDuration(ms.toLong)) - val serviceTimes = validTasks.map{case (info, metrics, exception) => - metrics.get.executorRunTime.toDouble} - val serviceQuantiles = "Duration" +: Distribution(serviceTimes).get.getQuantiles().map( - ms => parent.formatDuration(ms.toLong)) + val serviceTimes = validTasks.map { case TaskUIData(_, metrics, _) => + metrics.get.executorRunTime.toDouble + } + val serviceQuantiles = "Duration" +: Distribution(serviceTimes).get.getQuantiles() + .map(ms => parent.formatDuration(ms.toLong)) - val gettingResultTimes = validTasks.map{case (info, metrics, exception) => + val gettingResultTimes = validTasks.map { case TaskUIData(info, _, _) => if (info.gettingResultTime > 0) { (info.finishTime - info.gettingResultTime).toDouble } else { 0.0 } } - val gettingResultQuantiles = ("Time spent fetching task results" +: - Distribution(gettingResultTimes).get.getQuantiles().map( - millis => parent.formatDuration(millis.toLong))) + val gettingResultQuantiles = "Time spent fetching task results" +: + Distribution(gettingResultTimes).get.getQuantiles().map { millis => + parent.formatDuration(millis.toLong) + } // The scheduler delay includes the network delay to send the task to the worker // machine and to send back the result (but not the time to fetch the task result, // if it needed to be fetched from the block manager on the worker). - val schedulerDelays = validTasks.map{case (info, metrics, exception) => + val schedulerDelays = validTasks.map { case TaskUIData(info, metrics, _) => val totalExecutionTime = { if (info.gettingResultTime > 0) { (info.gettingResultTime - info.launchTime).toDouble @@ -149,35 +152,32 @@ private[spark] class StagePage(parent: JobProgressUI) { } totalExecutionTime - metrics.get.executorRunTime } - val schedulerDelayQuantiles = ("Scheduler delay" +: - Distribution(schedulerDelays).get.getQuantiles().map( - millis => parent.formatDuration(millis.toLong))) + val schedulerDelayQuantiles = "Scheduler delay" +: + Distribution(schedulerDelays).get.getQuantiles().map { millis => + parent.formatDuration(millis.toLong) + } def getQuantileCols(data: Seq[Double]) = Distribution(data).get.getQuantiles().map(d => Utils.bytesToString(d.toLong)) - val shuffleReadSizes = validTasks.map { - case(info, metrics, exception) => - metrics.get.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L).toDouble + val shuffleReadSizes = validTasks.map { case TaskUIData(_, metrics, _) => + metrics.get.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L).toDouble } val shuffleReadQuantiles = "Shuffle Read (Remote)" +: getQuantileCols(shuffleReadSizes) - val shuffleWriteSizes = validTasks.map { - case(info, metrics, exception) => - metrics.get.shuffleWriteMetrics.map(_.shuffleBytesWritten).getOrElse(0L).toDouble + val shuffleWriteSizes = validTasks.map { case TaskUIData(_, metrics, _) => + metrics.get.shuffleWriteMetrics.map(_.shuffleBytesWritten).getOrElse(0L).toDouble } val shuffleWriteQuantiles = "Shuffle Write" +: getQuantileCols(shuffleWriteSizes) - val memoryBytesSpilledSizes = validTasks.map { - case(info, metrics, exception) => - metrics.get.memoryBytesSpilled.toDouble + val memoryBytesSpilledSizes = validTasks.map { case TaskUIData(_, metrics, _) => + metrics.get.memoryBytesSpilled.toDouble } val memoryBytesSpilledQuantiles = "Shuffle spill (memory)" +: getQuantileCols(memoryBytesSpilledSizes) - val diskBytesSpilledSizes = validTasks.map { - case(info, metrics, exception) => - metrics.get.diskBytesSpilled.toDouble + val diskBytesSpilledSizes = validTasks.map { case TaskUIData(_, metrics, _) => + metrics.get.diskBytesSpilled.toDouble } val diskBytesSpilledQuantiles = "Shuffle spill (disk)" +: getQuantileCols(diskBytesSpilledSizes) @@ -195,98 +195,104 @@ private[spark] class StagePage(parent: JobProgressUI) { val quantileHeaders = Seq("Metric", "Min", "25th percentile", "Median", "75th percentile", "Max") def quantileRow(data: Seq[String]): Seq[Node] = {data.map(d => {d})} - Some(listingTable(quantileHeaders, quantileRow, listings, fixedWidth = true)) + Some(UIUtils.listingTable(quantileHeaders, quantileRow, listings, fixedWidth = true)) } - val executorTable = new ExecutorTable(parent, stageId) + val executorTable = new ExecutorTable(stageId, parent) val content = summary ++

    Summary Metrics for {numCompleted} Completed Tasks

    ++
    {summaryTable.getOrElse("No tasks have reported metrics yet.")}
    ++ -

    Aggregated Metrics by Executor

    ++ executorTable.toNodeSeq() ++ +

    Aggregated Metrics by Executor

    ++ executorTable.toNodeSeq ++

    Tasks

    ++ taskTable - headerSparkPage(content, parent.sc, "Details for Stage %d".format(stageId), Stages) + UIUtils.headerSparkPage( + content, basePath, appName, "Details for Stage %d".format(stageId), Stages) } } def taskRow(shuffleRead: Boolean, shuffleWrite: Boolean, bytesSpilled: Boolean) - (taskData: (TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])): Seq[Node] = { + (taskData: TaskUIData): Seq[Node] = { def fmtStackTrace(trace: Seq[StackTraceElement]): Seq[Node] = trace.map(e => {e.toString}) - val (info, metrics, exception) = taskData - - val duration = if (info.status == "RUNNING") info.timeRunning(System.currentTimeMillis()) - else metrics.map(m => m.executorRunTime).getOrElse(1) - val formatDuration = if (info.status == "RUNNING") parent.formatDuration(duration) - else metrics.map(m => parent.formatDuration(m.executorRunTime)).getOrElse("") - val gcTime = metrics.map(m => m.jvmGCTime).getOrElse(0L) - val serializationTime = metrics.map(m => m.resultSerializationTime).getOrElse(0L) - - val maybeShuffleRead = metrics.flatMap(m => m.shuffleReadMetrics).map(s => s.remoteBytesRead) - val shuffleReadSortable = maybeShuffleRead.map(_.toString).getOrElse("") - val shuffleReadReadable = maybeShuffleRead.map(Utils.bytesToString).getOrElse("") - - val maybeShuffleWrite = - metrics.flatMap{m => m.shuffleWriteMetrics}.map(s => s.shuffleBytesWritten) - val shuffleWriteSortable = maybeShuffleWrite.map(_.toString).getOrElse("") - val shuffleWriteReadable = maybeShuffleWrite.map(Utils.bytesToString).getOrElse("") - - val maybeWriteTime = metrics.flatMap(m => m.shuffleWriteMetrics).map(s => s.shuffleWriteTime) - val writeTimeSortable = maybeWriteTime.map(_.toString).getOrElse("") - val writeTimeReadable = maybeWriteTime.map( t => t / (1000 * 1000)).map{ ms => - if (ms == 0) "" else parent.formatDuration(ms)}.getOrElse("") - - val maybeMemoryBytesSpilled = metrics.map(m => m.memoryBytesSpilled) - val memoryBytesSpilledSortable = maybeMemoryBytesSpilled.map(_.toString).getOrElse("") - val memoryBytesSpilledReadable = maybeMemoryBytesSpilled.map(Utils.bytesToString).getOrElse("") - - val maybeDiskBytesSpilled = metrics.map{m => m.diskBytesSpilled} - val diskBytesSpilledSortable = maybeDiskBytesSpilled.map(_.toString).getOrElse("") - val diskBytesSpilledReadable = maybeDiskBytesSpilled.map(Utils.bytesToString).getOrElse("") - - - {info.index} - {info.taskId} - {info.status} - {info.taskLocality} - {info.host} - {dateFmt.format(new Date(info.launchTime))} - - {formatDuration} - - - {if (gcTime > 0) parent.formatDuration(gcTime) else ""} - - - {if (serializationTime > 0) parent.formatDuration(serializationTime) else ""} - - {if (shuffleRead) { - - {shuffleReadReadable} - - }} - {if (shuffleWrite) { - - {writeTimeReadable} - - - {shuffleWriteReadable} - - }} - {if (bytesSpilled) { - - {memoryBytesSpilledReadable} + + taskData match { case TaskUIData(info, metrics, exception) => + val duration = if (info.status == "RUNNING") info.timeRunning(System.currentTimeMillis()) + else metrics.map(_.executorRunTime).getOrElse(1L) + val formatDuration = if (info.status == "RUNNING") parent.formatDuration(duration) + else metrics.map(m => parent.formatDuration(m.executorRunTime)).getOrElse("") + val gcTime = metrics.map(_.jvmGCTime).getOrElse(0L) + val serializationTime = metrics.map(_.resultSerializationTime).getOrElse(0L) + + val maybeShuffleRead = metrics.flatMap(_.shuffleReadMetrics).map(_.remoteBytesRead) + val shuffleReadSortable = maybeShuffleRead.map(_.toString).getOrElse("") + val shuffleReadReadable = maybeShuffleRead.map(Utils.bytesToString).getOrElse("") + + val maybeShuffleWrite = + metrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleBytesWritten) + val shuffleWriteSortable = maybeShuffleWrite.map(_.toString).getOrElse("") + val shuffleWriteReadable = maybeShuffleWrite.map(Utils.bytesToString).getOrElse("") + + val maybeWriteTime = metrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleWriteTime) + val writeTimeSortable = maybeWriteTime.map(_.toString).getOrElse("") + val writeTimeReadable = maybeWriteTime.map( t => t / (1000 * 1000)).map { ms => + if (ms == 0) "" else parent.formatDuration(ms) + }.getOrElse("") + + val maybeMemoryBytesSpilled = metrics.map(_.memoryBytesSpilled) + val memoryBytesSpilledSortable = maybeMemoryBytesSpilled.map(_.toString).getOrElse("") + val memoryBytesSpilledReadable = + maybeMemoryBytesSpilled.map(Utils.bytesToString).getOrElse("") + + val maybeDiskBytesSpilled = metrics.map(_.diskBytesSpilled) + val diskBytesSpilledSortable = maybeDiskBytesSpilled.map(_.toString).getOrElse("") + val diskBytesSpilledReadable = maybeDiskBytesSpilled.map(Utils.bytesToString).getOrElse("") + + + {info.index} + {info.taskId} + {info.status} + {info.taskLocality} + {info.host} + {dateFmt.format(new Date(info.launchTime))} + + {formatDuration} + + + {if (gcTime > 0) parent.formatDuration(gcTime) else ""} - - {diskBytesSpilledReadable} + + {if (serializationTime > 0) parent.formatDuration(serializationTime) else ""} - }} - {exception.map(e => - - {e.className} ({e.description})
    - {fmtStackTrace(e.stackTrace)} -
    ).getOrElse("")} - - + {if (shuffleRead) { + + {shuffleReadReadable} + + }} + {if (shuffleWrite) { + + {writeTimeReadable} + + + {shuffleWriteReadable} + + }} + {if (bytesSpilled) { + + {memoryBytesSpilledReadable} + + + {diskBytesSpilledReadable} + + }} + + {exception.map { e => + + {e.className} ({e.description})
    + {fmtStackTrace(e.stackTrace)} +
    + }.getOrElse("")} + + + } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 99828487f1902..68fef5234c9ab 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -19,27 +19,27 @@ package org.apache.spark.ui.jobs import java.util.Date -import scala.collection.mutable.HashSet +import scala.collection.mutable.HashMap import scala.xml.Node -import org.apache.spark.scheduler.{SchedulingMode, StageInfo, TaskInfo} +import org.apache.spark.scheduler.{StageInfo, TaskInfo} import org.apache.spark.ui.UIUtils import org.apache.spark.util.Utils /** Page showing list of all ongoing and recently finished stages */ -private[spark] class StageTable(val stages: Seq[StageInfo], val parent: JobProgressUI) { +private[ui] class StageTable(stages: Seq[StageInfo], parent: JobProgressUI) { + private val basePath = parent.basePath + private val dateFmt = parent.dateFmt + private lazy val listener = parent.listener + private lazy val isFairScheduler = parent.isFairScheduler - val listener = parent.listener - val dateFmt = parent.dateFmt - val isFairScheduler = listener.sc.getSchedulingMode == SchedulingMode.FAIR - - def toNodeSeq(): Seq[Node] = { + def toNodeSeq: Seq[Node] = { listener.synchronized { stageTable(stageRow, stages) } } - /** Special table which merges two header cells. */ + /** Special table that merges two header cells. */ private def stageTable[T](makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { @@ -72,55 +72,56 @@ private[spark] class StageTable(val stages: Seq[StageInfo], val parent: JobProgr } - + /** Render an HTML row that represents a stage */ private def stageRow(s: StageInfo): Seq[Node] = { + val poolName = listener.stageIdToPool.get(s.stageId) + val nameLink = + + {s.name} + + val description = listener.stageIdToDescription.get(s.stageId) + .map(d =>
    {d}
    {nameLink}
    ).getOrElse(nameLink) val submissionTime = s.submissionTime match { case Some(t) => dateFmt.format(new Date(t)) case None => "Unknown" } - + val finishTime = s.completionTime.getOrElse(System.currentTimeMillis) + val duration = s.submissionTime.map { t => + if (finishTime > t) finishTime - t else System.currentTimeMillis - t + } + val formattedDuration = duration.map(d => parent.formatDuration(d)).getOrElse("Unknown") + val startedTasks = + listener.stageIdToTasksActive.getOrElse(s.stageId, HashMap[Long, TaskInfo]()).size + val completedTasks = listener.stageIdToTasksComplete.getOrElse(s.stageId, 0) + val failedTasks = listener.stageIdToTasksFailed.getOrElse(s.stageId, 0) match { + case f if f > 0 => "(%s failed)".format(f) + case _ => "" + } + val totalTasks = s.numTasks val shuffleReadSortable = listener.stageIdToShuffleRead.getOrElse(s.stageId, 0L) val shuffleRead = shuffleReadSortable match { case 0 => "" case b => Utils.bytesToString(b) } - val shuffleWriteSortable = listener.stageIdToShuffleWrite.getOrElse(s.stageId, 0L) val shuffleWrite = shuffleWriteSortable match { case 0 => "" case b => Utils.bytesToString(b) } - val startedTasks = listener.stageIdToTasksActive.getOrElse(s.stageId, HashSet[TaskInfo]()).size - val completedTasks = listener.stageIdToTasksComplete.getOrElse(s.stageId, 0) - val failedTasks = listener.stageIdToTasksFailed.getOrElse(s.stageId, 0) match { - case f if f > 0 => "(%s failed)".format(f) - case _ => "" - } - val totalTasks = s.numTasks - - val poolName = listener.stageIdToPool.get(s.stageId) - - val nameLink = - {s.name} - val description = listener.stageIdToDescription.get(s.stageId) - .map(d =>
    {d}
    {nameLink}
    ).getOrElse(nameLink) - val finishTime = s.completionTime.getOrElse(System.currentTimeMillis()) - val duration = s.submissionTime.map { t => - if (finishTime > t) finishTime - t else System.currentTimeMillis - t - } - {if (isFairScheduler) { - } - } + + }} - + diff --git a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala index cb2083eb019bf..4d8b01dbe6e1b 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala @@ -19,22 +19,80 @@ package org.apache.spark.ui.storage import javax.servlet.http.HttpServletRequest +import scala.collection.mutable + import org.eclipse.jetty.servlet.ServletContextHandler -import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.ui._ import org.apache.spark.ui.JettyUtils._ +import org.apache.spark.scheduler._ +import org.apache.spark.storage.{RDDInfo, StorageStatusListener, StorageUtils} /** Web UI showing storage status of all RDD's in the given SparkContext. */ -private[spark] class BlockManagerUI(val sc: SparkContext) extends Logging { - val indexPage = new IndexPage(this) - val rddPage = new RDDPage(this) +private[ui] class BlockManagerUI(parent: SparkUI) { + val appName = parent.appName + val basePath = parent.basePath + + private val indexPage = new IndexPage(this) + private val rddPage = new RDDPage(this) + private var _listener: Option[BlockManagerListener] = None + + lazy val listener = _listener.get + + def start() { + _listener = Some(new BlockManagerListener(parent.storageStatusListener)) + } def getHandlers = Seq[ServletContextHandler]( createServletHandler("/storage/rdd", - createServlet((request: HttpServletRequest) => rddPage.render(request), - sc.env.securityManager)), + (request: HttpServletRequest) => rddPage.render(request), parent.securityManager, basePath), createServletHandler("/storage", - createServlet((request: HttpServletRequest) => indexPage.render(request), - sc.env.securityManager)) + (request: HttpServletRequest) => indexPage.render(request), parent.securityManager, basePath) ) } + +/** + * A SparkListener that prepares information to be displayed on the BlockManagerUI + */ +private[ui] class BlockManagerListener(storageStatusListener: StorageStatusListener) + extends SparkListener { + + private val _rddInfoMap = mutable.Map[Int, RDDInfo]() + + def storageStatusList = storageStatusListener.storageStatusList + + /** Filter RDD info to include only those with cached partitions */ + def rddInfoList = _rddInfoMap.values.filter(_.numCachedPartitions > 0).toSeq + + /** Update each RDD's info to reflect any updates to the RDD's storage status */ + private def updateRDDInfo() { + val rddInfos = _rddInfoMap.values.toSeq + val updatedRddInfos = StorageUtils.rddInfoFromStorageStatus(storageStatusList, rddInfos) + updatedRddInfos.foreach { info => _rddInfoMap(info.id) = info } + } + + /** + * Assumes the storage status list is fully up-to-date. This implies the corresponding + * StorageStatusSparkListener must process the SparkListenerTaskEnd event before this listener. + */ + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { + val metrics = taskEnd.taskMetrics + if (metrics != null && metrics.updatedBlocks.isDefined) { + updateRDDInfo() + } + } + + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = synchronized { + val rddInfo = stageSubmitted.stageInfo.rddInfo + _rddInfoMap(rddInfo.id) = rddInfo + } + + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = synchronized { + // Remove all partitions that are no longer cached + _rddInfoMap.retain { case (_, info) => info.numCachedPartitions > 0 } + } + + override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD) = synchronized { + updateRDDInfo() + } +} diff --git a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala index 6a3c41fb1155d..b2732de51058a 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala @@ -21,36 +21,37 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.storage.{RDDInfo, StorageUtils} +import org.apache.spark.storage.RDDInfo import org.apache.spark.ui.Page._ -import org.apache.spark.ui.UIUtils._ +import org.apache.spark.ui.UIUtils import org.apache.spark.util.Utils /** Page showing list of RDD's currently stored in the cluster */ -private[spark] class IndexPage(parent: BlockManagerUI) { - val sc = parent.sc +private[ui] class IndexPage(parent: BlockManagerUI) { + private val appName = parent.appName + private val basePath = parent.basePath + private lazy val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { - val storageStatusList = sc.getExecutorStorageStatus - // Calculate macro-level statistics - - val rddHeaders = Seq( - "RDD Name", - "Storage Level", - "Cached Partitions", - "Fraction Cached", - "Size in Memory", - "Size on Disk") - val rdds = StorageUtils.rddInfoFromStorageStatus(storageStatusList, sc) - val content = listingTable(rddHeaders, rddRow, rdds) - - headerSparkPage(content, parent.sc, "Storage ", Storage) + val rdds = listener.rddInfoList + val content = UIUtils.listingTable(rddHeader, rddRow, rdds) + UIUtils.headerSparkPage(content, basePath, appName, "Storage ", Storage) } - def rddRow(rdd: RDDInfo): Seq[Node] = { + /** Header fields for the RDD table */ + private def rddHeader = Seq( + "RDD Name", + "Storage Level", + "Cached Partitions", + "Fraction Cached", + "Size in Memory", + "Size on Disk") + + /** Render an HTML row representing an RDD */ + private def rddRow(rdd: RDDInfo): Seq[Node] = { diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index 78b149b14b1d4..3f42eba4ece00 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -21,36 +21,37 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.storage.{BlockId, StorageStatus, StorageUtils} -import org.apache.spark.storage.BlockManagerMasterActor.BlockStatus +import org.apache.spark.storage.{BlockId, BlockStatus, StorageStatus, StorageUtils} import org.apache.spark.ui.Page._ -import org.apache.spark.ui.UIUtils._ +import org.apache.spark.ui.UIUtils import org.apache.spark.util.Utils /** Page showing storage details for a given RDD */ -private[spark] class RDDPage(parent: BlockManagerUI) { - val sc = parent.sc +private[ui] class RDDPage(parent: BlockManagerUI) { + private val appName = parent.appName + private val basePath = parent.basePath + private lazy val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { - val id = request.getParameter("id").toInt - val storageStatusList = sc.getExecutorStorageStatus - val filteredStorageStatusList = StorageUtils.filterStorageStatusByRDD(storageStatusList, id) - val rddInfo = StorageUtils.rddInfoFromStorageStatus(filteredStorageStatusList, sc).head - - val workerHeaders = Seq("Host", "Memory Usage", "Disk Usage") - val workers = filteredStorageStatusList.map((id, _)) - val workerTable = listingTable(workerHeaders, workerRow, workers) + val rddId = request.getParameter("id").toInt + val storageStatusList = listener.storageStatusList + val rddInfo = listener.rddInfoList.find(_.id == rddId).getOrElse { + // Rather than crashing, render an "RDD Not Found" page + return UIUtils.headerSparkPage(Seq[Node](), basePath, appName, "RDD Not Found", Storage) + } - val blockHeaders = Seq("Block Name", "Storage Level", "Size in Memory", "Size on Disk", - "Executors") + // Worker table + val workers = storageStatusList.map((rddId, _)) + val workerTable = UIUtils.listingTable(workerHeader, workerRow, workers) - val blockStatuses = filteredStorageStatusList.flatMap(_.blocks).toArray. - sortWith(_._1.name < _._1.name) + // Block table + val filteredStorageStatusList = StorageUtils.filterStorageStatusByRDD(storageStatusList, rddId) + val blockStatuses = filteredStorageStatusList.flatMap(_.blocks).sortWith(_._1.name < _._1.name) val blockLocations = StorageUtils.blockLocationsFromStorageStatus(filteredStorageStatusList) - val blocks = blockStatuses.map { - case(id, status) => (id, status, blockLocations.get(id).getOrElse(Seq("UNKNOWN"))) + val blocks = blockStatuses.map { case (blockId, status) => + (blockId, status, blockLocations.get(blockId).getOrElse(Seq[String]("Unknown"))) } - val blockTable = listingTable(blockHeaders, blockRow, blocks) + val blockTable = UIUtils.listingTable(blockHeader, blockRow, blocks) val content =
    @@ -94,10 +95,39 @@ private[spark] class RDDPage(parent: BlockManagerUI) {
    ; - headerSparkPage(content, parent.sc, "RDD Storage Info for " + rddInfo.name, Storage) + UIUtils.headerSparkPage( + content, basePath, appName, "RDD Storage Info for " + rddInfo.name, Storage) + } + + /** Header fields for the worker table */ + private def workerHeader = Seq( + "Host", + "Memory Usage", + "Disk Usage") + + /** Header fields for the block table */ + private def blockHeader = Seq( + "Block Name", + "Storage Level", + "Size in Memory", + "Size on Disk", + "Executors") + + /** Render an HTML row representing a worker */ + private def workerRow(worker: (Int, StorageStatus)): Seq[Node] = { + val (rddId, status) = worker + + + + + } - def blockRow(row: (BlockId, BlockStatus, Seq[String])): Seq[Node] = { + /** Render an HTML row representing a block */ + private def blockRow(row: (BlockId, BlockStatus, Seq[String])): Seq[Node] = { val (id, block, locations) = row @@ -115,16 +145,4 @@ private[spark] class RDDPage(parent: BlockManagerUI) { } - - def workerRow(worker: (Int, StorageStatus)): Seq[Node] = { - val (rddId, status) = worker - - - - - - } } diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala new file mode 100644 index 0000000000000..f07962096a32c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -0,0 +1,165 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import java.io._ +import java.net.URI +import java.text.SimpleDateFormat +import java.util.Date + +import it.unimi.dsi.fastutil.io.FastBufferedOutputStream +import org.apache.hadoop.fs.{FSDataOutputStream, Path} + +import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.io.CompressionCodec + +/** + * A generic class for logging information to file. + * + * @param logDir Path to the directory in which files are logged + * @param outputBufferSize The buffer size to use when writing to an output stream in bytes + * @param compress Whether to compress output + * @param overwrite Whether to overwrite existing files + */ +class FileLogger( + logDir: String, + conf: SparkConf = new SparkConf, + outputBufferSize: Int = 8 * 1024, // 8 KB + compress: Boolean = false, + overwrite: Boolean = true) + extends Logging { + + private val DATE_FORMAT = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + private val fileSystem = Utils.getHadoopFileSystem(new URI(logDir)) + private var fileIndex = 0 + + // Only used if compression is enabled + private lazy val compressionCodec = CompressionCodec.createCodec(conf) + + // Only defined if the file system scheme is not local + private var hadoopDataStream: Option[FSDataOutputStream] = None + + private var writer: Option[PrintWriter] = { + createLogDir() + Some(createWriter()) + } + + /** + * Create a logging directory with the given path. + */ + private def createLogDir() { + val path = new Path(logDir) + if (fileSystem.exists(path)) { + if (overwrite) { + logWarning("Log directory %s already exists. Overwriting...".format(logDir)) + // Second parameter is whether to delete recursively + fileSystem.delete(path, true) + } else { + throw new IOException("Log directory %s already exists!".format(logDir)) + } + } + if (!fileSystem.mkdirs(path)) { + throw new IOException("Error in creating log directory: %s".format(logDir)) + } + } + + /** + * Create a new writer for the file identified by the given path. + */ + private def createWriter(): PrintWriter = { + val logPath = logDir + "/" + fileIndex + val uri = new URI(logPath) + + /* The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844). + * Therefore, for local files, use FileOutputStream instead. */ + val dstream = uri.getScheme match { + case "file" | null => + // Second parameter is whether to append + new FileOutputStream(logPath, !overwrite) + + case _ => + val path = new Path(logPath) + hadoopDataStream = Some(fileSystem.create(path, overwrite)) + hadoopDataStream.get + } + + val bstream = new FastBufferedOutputStream(dstream, outputBufferSize) + val cstream = if (compress) compressionCodec.compressedOutputStream(bstream) else bstream + new PrintWriter(cstream) + } + + /** + * Log the message to the given writer. + * @param msg The message to be logged + * @param withTime Whether to prepend message with a timestamp + */ + def log(msg: String, withTime: Boolean = false) { + val writeInfo = if (!withTime) msg else { + val date = new Date(System.currentTimeMillis()) + DATE_FORMAT.format(date) + ": " + msg + } + writer.foreach(_.print(writeInfo)) + } + + /** + * Log the message to the given writer as a new line. + * @param msg The message to be logged + * @param withTime Whether to prepend message with a timestamp + */ + def logLine(msg: String, withTime: Boolean = false) = log(msg + "\n", withTime) + + /** + * Flush the writer to disk manually. + * + * If the Hadoop FileSystem is used, the underlying FSDataOutputStream (r1.0.4) must be + * sync()'ed manually as it does not support flush(), which is invoked by when higher + * level streams are flushed. + */ + def flush() { + writer.foreach(_.flush()) + hadoopDataStream.foreach(_.sync()) + } + + /** + * Close the writer. Any subsequent calls to log or flush will have no effect. + */ + def close() { + writer.foreach(_.close()) + writer = None + } + + /** + * Start a writer for a new file if one does not already exit. + */ + def start() { + writer.getOrElse { + fileIndex += 1 + writer = Some(createWriter()) + } + } + + /** + * Close all open writers, streams, and file systems. Any subsequent uses of this FileLogger + * instance will throw exceptions. + */ + def stop() { + hadoopDataStream.foreach(_.close()) + writer.foreach(_.close()) + fileSystem.close() + } +} diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala new file mode 100644 index 0000000000000..346f2b7856791 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -0,0 +1,710 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import java.util.{Properties, UUID} + +import scala.collection.JavaConverters._ +import scala.collection.Map + +import org.json4s.DefaultFormats +import org.json4s.JsonDSL._ +import org.json4s.JsonAST._ + +import org.apache.spark.executor.{ShuffleReadMetrics, ShuffleWriteMetrics, TaskMetrics} +import org.apache.spark.scheduler._ +import org.apache.spark.storage._ +import org.apache.spark._ + +private[spark] object JsonProtocol { + private implicit val format = DefaultFormats + + /** ------------------------------------------------- * + * JSON serialization methods for SparkListenerEvents | + * -------------------------------------------------- */ + + def sparkEventToJson(event: SparkListenerEvent): JValue = { + event match { + case stageSubmitted: SparkListenerStageSubmitted => + stageSubmittedToJson(stageSubmitted) + case stageCompleted: SparkListenerStageCompleted => + stageCompletedToJson(stageCompleted) + case taskStart: SparkListenerTaskStart => + taskStartToJson(taskStart) + case taskGettingResult: SparkListenerTaskGettingResult => + taskGettingResultToJson(taskGettingResult) + case taskEnd: SparkListenerTaskEnd => + taskEndToJson(taskEnd) + case jobStart: SparkListenerJobStart => + jobStartToJson(jobStart) + case jobEnd: SparkListenerJobEnd => + jobEndToJson(jobEnd) + case environmentUpdate: SparkListenerEnvironmentUpdate => + environmentUpdateToJson(environmentUpdate) + case blockManagerAdded: SparkListenerBlockManagerAdded => + blockManagerAddedToJson(blockManagerAdded) + case blockManagerRemoved: SparkListenerBlockManagerRemoved => + blockManagerRemovedToJson(blockManagerRemoved) + case unpersistRDD: SparkListenerUnpersistRDD => + unpersistRDDToJson(unpersistRDD) + + // Not used, but keeps compiler happy + case SparkListenerShutdown => JNothing + } + } + + def stageSubmittedToJson(stageSubmitted: SparkListenerStageSubmitted): JValue = { + val stageInfo = stageInfoToJson(stageSubmitted.stageInfo) + val properties = propertiesToJson(stageSubmitted.properties) + ("Event" -> Utils.getFormattedClassName(stageSubmitted)) ~ + ("Stage Info" -> stageInfo) ~ + ("Properties" -> properties) + } + + def stageCompletedToJson(stageCompleted: SparkListenerStageCompleted): JValue = { + val stageInfo = stageInfoToJson(stageCompleted.stageInfo) + ("Event" -> Utils.getFormattedClassName(stageCompleted)) ~ + ("Stage Info" -> stageInfo) + } + + def taskStartToJson(taskStart: SparkListenerTaskStart): JValue = { + val taskInfo = taskStart.taskInfo + val taskInfoJson = if (taskInfo != null) taskInfoToJson(taskInfo) else JNothing + ("Event" -> Utils.getFormattedClassName(taskStart)) ~ + ("Stage ID" -> taskStart.stageId) ~ + ("Task Info" -> taskInfoJson) + } + + def taskGettingResultToJson(taskGettingResult: SparkListenerTaskGettingResult): JValue = { + val taskInfo = taskGettingResult.taskInfo + val taskInfoJson = if (taskInfo != null) taskInfoToJson(taskInfo) else JNothing + ("Event" -> Utils.getFormattedClassName(taskGettingResult)) ~ + ("Task Info" -> taskInfoJson) + } + + def taskEndToJson(taskEnd: SparkListenerTaskEnd): JValue = { + val taskEndReason = taskEndReasonToJson(taskEnd.reason) + val taskInfo = taskEnd.taskInfo + val taskInfoJson = if (taskInfo != null) taskInfoToJson(taskInfo) else JNothing + val taskMetrics = taskEnd.taskMetrics + val taskMetricsJson = if (taskMetrics != null) taskMetricsToJson(taskMetrics) else JNothing + ("Event" -> Utils.getFormattedClassName(taskEnd)) ~ + ("Stage ID" -> taskEnd.stageId) ~ + ("Task Type" -> taskEnd.taskType) ~ + ("Task End Reason" -> taskEndReason) ~ + ("Task Info" -> taskInfoJson) ~ + ("Task Metrics" -> taskMetricsJson) + } + + def jobStartToJson(jobStart: SparkListenerJobStart): JValue = { + val properties = propertiesToJson(jobStart.properties) + ("Event" -> Utils.getFormattedClassName(jobStart)) ~ + ("Job ID" -> jobStart.jobId) ~ + ("Stage IDs" -> jobStart.stageIds) ~ + ("Properties" -> properties) + } + + def jobEndToJson(jobEnd: SparkListenerJobEnd): JValue = { + val jobResult = jobResultToJson(jobEnd.jobResult) + ("Event" -> Utils.getFormattedClassName(jobEnd)) ~ + ("Job ID" -> jobEnd.jobId) ~ + ("Job Result" -> jobResult) + } + + def environmentUpdateToJson(environmentUpdate: SparkListenerEnvironmentUpdate): JValue = { + val environmentDetails = environmentUpdate.environmentDetails + val jvmInformation = mapToJson(environmentDetails("JVM Information").toMap) + val sparkProperties = mapToJson(environmentDetails("Spark Properties").toMap) + val systemProperties = mapToJson(environmentDetails("System Properties").toMap) + val classpathEntries = mapToJson(environmentDetails("Classpath Entries").toMap) + ("Event" -> Utils.getFormattedClassName(environmentUpdate)) ~ + ("JVM Information" -> jvmInformation) ~ + ("Spark Properties" -> sparkProperties) ~ + ("System Properties" -> systemProperties) ~ + ("Classpath Entries" -> classpathEntries) + } + + def blockManagerAddedToJson(blockManagerAdded: SparkListenerBlockManagerAdded): JValue = { + val blockManagerId = blockManagerIdToJson(blockManagerAdded.blockManagerId) + ("Event" -> Utils.getFormattedClassName(blockManagerAdded)) ~ + ("Block Manager ID" -> blockManagerId) ~ + ("Maximum Memory" -> blockManagerAdded.maxMem) + } + + def blockManagerRemovedToJson(blockManagerRemoved: SparkListenerBlockManagerRemoved): JValue = { + val blockManagerId = blockManagerIdToJson(blockManagerRemoved.blockManagerId) + ("Event" -> Utils.getFormattedClassName(blockManagerRemoved)) ~ + ("Block Manager ID" -> blockManagerId) + } + + def unpersistRDDToJson(unpersistRDD: SparkListenerUnpersistRDD): JValue = { + ("Event" -> Utils.getFormattedClassName(unpersistRDD)) ~ + ("RDD ID" -> unpersistRDD.rddId) + } + + + /** ------------------------------------------------------------------- * + * JSON serialization methods for classes SparkListenerEvents depend on | + * -------------------------------------------------------------------- */ + + def stageInfoToJson(stageInfo: StageInfo): JValue = { + val rddInfo = rddInfoToJson(stageInfo.rddInfo) + val submissionTime = stageInfo.submissionTime.map(JInt(_)).getOrElse(JNothing) + val completionTime = stageInfo.completionTime.map(JInt(_)).getOrElse(JNothing) + ("Stage ID" -> stageInfo.stageId) ~ + ("Stage Name" -> stageInfo.name) ~ + ("Number of Tasks" -> stageInfo.numTasks) ~ + ("RDD Info" -> rddInfo) ~ + ("Submission Time" -> submissionTime) ~ + ("Completion Time" -> completionTime) ~ + ("Emitted Task Size Warning" -> stageInfo.emittedTaskSizeWarning) + } + + def taskInfoToJson(taskInfo: TaskInfo): JValue = { + ("Task ID" -> taskInfo.taskId) ~ + ("Index" -> taskInfo.index) ~ + ("Launch Time" -> taskInfo.launchTime) ~ + ("Executor ID" -> taskInfo.executorId) ~ + ("Host" -> taskInfo.host) ~ + ("Locality" -> taskInfo.taskLocality.toString) ~ + ("Getting Result Time" -> taskInfo.gettingResultTime) ~ + ("Finish Time" -> taskInfo.finishTime) ~ + ("Failed" -> taskInfo.failed) ~ + ("Serialized Size" -> taskInfo.serializedSize) + } + + def taskMetricsToJson(taskMetrics: TaskMetrics): JValue = { + val shuffleReadMetrics = + taskMetrics.shuffleReadMetrics.map(shuffleReadMetricsToJson).getOrElse(JNothing) + val shuffleWriteMetrics = + taskMetrics.shuffleWriteMetrics.map(shuffleWriteMetricsToJson).getOrElse(JNothing) + val updatedBlocks = taskMetrics.updatedBlocks.map { blocks => + JArray(blocks.toList.map { case (id, status) => + ("Block ID" -> blockIdToJson(id)) ~ + ("Status" -> blockStatusToJson(status)) + }) + }.getOrElse(JNothing) + ("Host Name" -> taskMetrics.hostname) ~ + ("Executor Deserialize Time" -> taskMetrics.executorDeserializeTime) ~ + ("Executor Run Time" -> taskMetrics.executorRunTime) ~ + ("Result Size" -> taskMetrics.resultSize) ~ + ("JVM GC Time" -> taskMetrics.jvmGCTime) ~ + ("Result Serialization Time" -> taskMetrics.resultSerializationTime) ~ + ("Memory Bytes Spilled" -> taskMetrics.memoryBytesSpilled) ~ + ("Disk Bytes Spilled" -> taskMetrics.diskBytesSpilled) ~ + ("Shuffle Read Metrics" -> shuffleReadMetrics) ~ + ("Shuffle Write Metrics" -> shuffleWriteMetrics) ~ + ("Updated Blocks" -> updatedBlocks) + } + + def shuffleReadMetricsToJson(shuffleReadMetrics: ShuffleReadMetrics): JValue = { + ("Shuffle Finish Time" -> shuffleReadMetrics.shuffleFinishTime) ~ + ("Total Blocks Fetched" -> shuffleReadMetrics.totalBlocksFetched) ~ + ("Remote Blocks Fetched" -> shuffleReadMetrics.remoteBlocksFetched) ~ + ("Local Blocks Fetched" -> shuffleReadMetrics.localBlocksFetched) ~ + ("Fetch Wait Time" -> shuffleReadMetrics.fetchWaitTime) ~ + ("Remote Bytes Read" -> shuffleReadMetrics.remoteBytesRead) + } + + def shuffleWriteMetricsToJson(shuffleWriteMetrics: ShuffleWriteMetrics): JValue = { + ("Shuffle Bytes Written" -> shuffleWriteMetrics.shuffleBytesWritten) ~ + ("Shuffle Write Time" -> shuffleWriteMetrics.shuffleWriteTime) + } + + def taskEndReasonToJson(taskEndReason: TaskEndReason): JValue = { + val reason = Utils.getFormattedClassName(taskEndReason) + val json = taskEndReason match { + case fetchFailed: FetchFailed => + val blockManagerAddress = blockManagerIdToJson(fetchFailed.bmAddress) + ("Block Manager Address" -> blockManagerAddress) ~ + ("Shuffle ID" -> fetchFailed.shuffleId) ~ + ("Map ID" -> fetchFailed.mapId) ~ + ("Reduce ID" -> fetchFailed.reduceId) + case exceptionFailure: ExceptionFailure => + val stackTrace = stackTraceToJson(exceptionFailure.stackTrace) + val metrics = exceptionFailure.metrics.map(taskMetricsToJson).getOrElse(JNothing) + ("Class Name" -> exceptionFailure.className) ~ + ("Description" -> exceptionFailure.description) ~ + ("Stack Trace" -> stackTrace) ~ + ("Metrics" -> metrics) + case _ => Utils.emptyJson + } + ("Reason" -> reason) ~ json + } + + def blockManagerIdToJson(blockManagerId: BlockManagerId): JValue = { + ("Executor ID" -> blockManagerId.executorId) ~ + ("Host" -> blockManagerId.host) ~ + ("Port" -> blockManagerId.port) ~ + ("Netty Port" -> blockManagerId.nettyPort) + } + + def jobResultToJson(jobResult: JobResult): JValue = { + val result = Utils.getFormattedClassName(jobResult) + val json = jobResult match { + case JobSucceeded => Utils.emptyJson + case jobFailed: JobFailed => + val exception = exceptionToJson(jobFailed.exception) + ("Exception" -> exception) ~ + ("Failed Stage ID" -> jobFailed.failedStageId) + } + ("Result" -> result) ~ json + } + + def rddInfoToJson(rddInfo: RDDInfo): JValue = { + val storageLevel = storageLevelToJson(rddInfo.storageLevel) + ("RDD ID" -> rddInfo.id) ~ + ("Name" -> rddInfo.name) ~ + ("Storage Level" -> storageLevel) ~ + ("Number of Partitions" -> rddInfo.numPartitions) ~ + ("Number of Cached Partitions" -> rddInfo.numCachedPartitions) ~ + ("Memory Size" -> rddInfo.memSize) ~ + ("Disk Size" -> rddInfo.diskSize) + } + + def storageLevelToJson(storageLevel: StorageLevel): JValue = { + ("Use Disk" -> storageLevel.useDisk) ~ + ("Use Memory" -> storageLevel.useMemory) ~ + ("Deserialized" -> storageLevel.deserialized) ~ + ("Replication" -> storageLevel.replication) + } + + def blockIdToJson(blockId: BlockId): JValue = { + val blockType = Utils.getFormattedClassName(blockId) + val json: JObject = blockId match { + case rddBlockId: RDDBlockId => + ("RDD ID" -> rddBlockId.rddId) ~ + ("Split Index" -> rddBlockId.splitIndex) + case shuffleBlockId: ShuffleBlockId => + ("Shuffle ID" -> shuffleBlockId.shuffleId) ~ + ("Map ID" -> shuffleBlockId.mapId) ~ + ("Reduce ID" -> shuffleBlockId.reduceId) + case broadcastBlockId: BroadcastBlockId => + "Broadcast ID" -> broadcastBlockId.broadcastId + case broadcastHelperBlockId: BroadcastHelperBlockId => + ("Broadcast Block ID" -> blockIdToJson(broadcastHelperBlockId.broadcastId)) ~ + ("Helper Type" -> broadcastHelperBlockId.hType) + case taskResultBlockId: TaskResultBlockId => + "Task ID" -> taskResultBlockId.taskId + case streamBlockId: StreamBlockId => + ("Stream ID" -> streamBlockId.streamId) ~ + ("Unique ID" -> streamBlockId.uniqueId) + case tempBlockId: TempBlockId => + val uuid = UUIDToJson(tempBlockId.id) + "Temp ID" -> uuid + case testBlockId: TestBlockId => + "Test ID" -> testBlockId.id + } + ("Type" -> blockType) ~ json + } + + def blockStatusToJson(blockStatus: BlockStatus): JValue = { + val storageLevel = storageLevelToJson(blockStatus.storageLevel) + ("Storage Level" -> storageLevel) ~ + ("Memory Size" -> blockStatus.memSize) ~ + ("Disk Size" -> blockStatus.diskSize) + } + + + /** ------------------------------ * + * Util JSON serialization methods | + * ------------------------------- */ + + def mapToJson(m: Map[String, String]): JValue = { + val jsonFields = m.map { case (k, v) => JField(k, JString(v)) } + JObject(jsonFields.toList) + } + + def propertiesToJson(properties: Properties): JValue = { + Option(properties).map { p => + mapToJson(p.asScala) + }.getOrElse(JNothing) + } + + def UUIDToJson(id: UUID): JValue = { + ("Least Significant Bits" -> id.getLeastSignificantBits) ~ + ("Most Significant Bits" -> id.getMostSignificantBits) + } + + def stackTraceToJson(stackTrace: Array[StackTraceElement]): JValue = { + JArray(stackTrace.map { case line => + ("Declaring Class" -> line.getClassName) ~ + ("Method Name" -> line.getMethodName) ~ + ("File Name" -> line.getFileName) ~ + ("Line Number" -> line.getLineNumber) + }.toList) + } + + def exceptionToJson(exception: Exception): JValue = { + ("Message" -> exception.getMessage) ~ + ("Stack Trace" -> stackTraceToJson(exception.getStackTrace)) + } + + + /** --------------------------------------------------- * + * JSON deserialization methods for SparkListenerEvents | + * ---------------------------------------------------- */ + + def sparkEventFromJson(json: JValue): SparkListenerEvent = { + val stageSubmitted = Utils.getFormattedClassName(SparkListenerStageSubmitted) + val stageCompleted = Utils.getFormattedClassName(SparkListenerStageCompleted) + val taskStart = Utils.getFormattedClassName(SparkListenerTaskStart) + val taskGettingResult = Utils.getFormattedClassName(SparkListenerTaskGettingResult) + val taskEnd = Utils.getFormattedClassName(SparkListenerTaskEnd) + val jobStart = Utils.getFormattedClassName(SparkListenerJobStart) + val jobEnd = Utils.getFormattedClassName(SparkListenerJobEnd) + val environmentUpdate = Utils.getFormattedClassName(SparkListenerEnvironmentUpdate) + val blockManagerAdded = Utils.getFormattedClassName(SparkListenerBlockManagerAdded) + val blockManagerRemoved = Utils.getFormattedClassName(SparkListenerBlockManagerRemoved) + val unpersistRDD = Utils.getFormattedClassName(SparkListenerUnpersistRDD) + + (json \ "Event").extract[String] match { + case `stageSubmitted` => stageSubmittedFromJson(json) + case `stageCompleted` => stageCompletedFromJson(json) + case `taskStart` => taskStartFromJson(json) + case `taskGettingResult` => taskGettingResultFromJson(json) + case `taskEnd` => taskEndFromJson(json) + case `jobStart` => jobStartFromJson(json) + case `jobEnd` => jobEndFromJson(json) + case `environmentUpdate` => environmentUpdateFromJson(json) + case `blockManagerAdded` => blockManagerAddedFromJson(json) + case `blockManagerRemoved` => blockManagerRemovedFromJson(json) + case `unpersistRDD` => unpersistRDDFromJson(json) + } + } + + def stageSubmittedFromJson(json: JValue): SparkListenerStageSubmitted = { + val stageInfo = stageInfoFromJson(json \ "Stage Info") + val properties = propertiesFromJson(json \ "Properties") + SparkListenerStageSubmitted(stageInfo, properties) + } + + def stageCompletedFromJson(json: JValue): SparkListenerStageCompleted = { + val stageInfo = stageInfoFromJson(json \ "Stage Info") + SparkListenerStageCompleted(stageInfo) + } + + def taskStartFromJson(json: JValue): SparkListenerTaskStart = { + val stageId = (json \ "Stage ID").extract[Int] + val taskInfo = taskInfoFromJson(json \ "Task Info") + SparkListenerTaskStart(stageId, taskInfo) + } + + def taskGettingResultFromJson(json: JValue): SparkListenerTaskGettingResult = { + val taskInfo = taskInfoFromJson(json \ "Task Info") + SparkListenerTaskGettingResult(taskInfo) + } + + def taskEndFromJson(json: JValue): SparkListenerTaskEnd = { + val stageId = (json \ "Stage ID").extract[Int] + val taskType = (json \ "Task Type").extract[String] + val taskEndReason = taskEndReasonFromJson(json \ "Task End Reason") + val taskInfo = taskInfoFromJson(json \ "Task Info") + val taskMetrics = taskMetricsFromJson(json \ "Task Metrics") + SparkListenerTaskEnd(stageId, taskType, taskEndReason, taskInfo, taskMetrics) + } + + def jobStartFromJson(json: JValue): SparkListenerJobStart = { + val jobId = (json \ "Job ID").extract[Int] + val stageIds = (json \ "Stage IDs").extract[List[JValue]].map(_.extract[Int]) + val properties = propertiesFromJson(json \ "Properties") + SparkListenerJobStart(jobId, stageIds, properties) + } + + def jobEndFromJson(json: JValue): SparkListenerJobEnd = { + val jobId = (json \ "Job ID").extract[Int] + val jobResult = jobResultFromJson(json \ "Job Result") + SparkListenerJobEnd(jobId, jobResult) + } + + def environmentUpdateFromJson(json: JValue): SparkListenerEnvironmentUpdate = { + val environmentDetails = Map[String, Seq[(String, String)]]( + "JVM Information" -> mapFromJson(json \ "JVM Information").toSeq, + "Spark Properties" -> mapFromJson(json \ "Spark Properties").toSeq, + "System Properties" -> mapFromJson(json \ "System Properties").toSeq, + "Classpath Entries" -> mapFromJson(json \ "Classpath Entries").toSeq) + SparkListenerEnvironmentUpdate(environmentDetails) + } + + def blockManagerAddedFromJson(json: JValue): SparkListenerBlockManagerAdded = { + val blockManagerId = blockManagerIdFromJson(json \ "Block Manager ID") + val maxMem = (json \ "Maximum Memory").extract[Long] + SparkListenerBlockManagerAdded(blockManagerId, maxMem) + } + + def blockManagerRemovedFromJson(json: JValue): SparkListenerBlockManagerRemoved = { + val blockManagerId = blockManagerIdFromJson(json \ "Block Manager ID") + SparkListenerBlockManagerRemoved(blockManagerId) + } + + def unpersistRDDFromJson(json: JValue): SparkListenerUnpersistRDD = { + SparkListenerUnpersistRDD((json \ "RDD ID").extract[Int]) + } + + + /** --------------------------------------------------------------------- * + * JSON deserialization methods for classes SparkListenerEvents depend on | + * ---------------------------------------------------------------------- */ + + def stageInfoFromJson(json: JValue): StageInfo = { + val stageId = (json \ "Stage ID").extract[Int] + val stageName = (json \ "Stage Name").extract[String] + val numTasks = (json \ "Number of Tasks").extract[Int] + val rddInfo = rddInfoFromJson(json \ "RDD Info") + val submissionTime = Utils.jsonOption(json \ "Submission Time").map(_.extract[Long]) + val completionTime = Utils.jsonOption(json \ "Completion Time").map(_.extract[Long]) + val emittedTaskSizeWarning = (json \ "Emitted Task Size Warning").extract[Boolean] + + val stageInfo = new StageInfo(stageId, stageName, numTasks, rddInfo) + stageInfo.submissionTime = submissionTime + stageInfo.completionTime = completionTime + stageInfo.emittedTaskSizeWarning = emittedTaskSizeWarning + stageInfo + } + + def taskInfoFromJson(json: JValue): TaskInfo = { + val taskId = (json \ "Task ID").extract[Long] + val index = (json \ "Index").extract[Int] + val launchTime = (json \ "Launch Time").extract[Long] + val executorId = (json \ "Executor ID").extract[String] + val host = (json \ "Host").extract[String] + val taskLocality = TaskLocality.withName((json \ "Locality").extract[String]) + val gettingResultTime = (json \ "Getting Result Time").extract[Long] + val finishTime = (json \ "Finish Time").extract[Long] + val failed = (json \ "Failed").extract[Boolean] + val serializedSize = (json \ "Serialized Size").extract[Int] + + val taskInfo = new TaskInfo(taskId, index, launchTime, executorId, host, taskLocality) + taskInfo.gettingResultTime = gettingResultTime + taskInfo.finishTime = finishTime + taskInfo.failed = failed + taskInfo.serializedSize = serializedSize + taskInfo + } + + def taskMetricsFromJson(json: JValue): TaskMetrics = { + val metrics = new TaskMetrics + metrics.hostname = (json \ "Host Name").extract[String] + metrics.executorDeserializeTime = (json \ "Executor Deserialize Time").extract[Long] + metrics.executorRunTime = (json \ "Executor Run Time").extract[Long] + metrics.resultSize = (json \ "Result Size").extract[Long] + metrics.jvmGCTime = (json \ "JVM GC Time").extract[Long] + metrics.resultSerializationTime = (json \ "Result Serialization Time").extract[Long] + metrics.memoryBytesSpilled = (json \ "Memory Bytes Spilled").extract[Long] + metrics.diskBytesSpilled = (json \ "Disk Bytes Spilled").extract[Long] + metrics.shuffleReadMetrics = + Utils.jsonOption(json \ "Shuffle Read Metrics").map(shuffleReadMetricsFromJson) + metrics.shuffleWriteMetrics = + Utils.jsonOption(json \ "Shuffle Write Metrics").map(shuffleWriteMetricsFromJson) + metrics.updatedBlocks = Utils.jsonOption(json \ "Updated Blocks").map { value => + value.extract[List[JValue]].map { block => + val id = blockIdFromJson(block \ "Block ID") + val status = blockStatusFromJson(block \ "Status") + (id, status) + } + } + metrics + } + + def shuffleReadMetricsFromJson(json: JValue): ShuffleReadMetrics = { + val metrics = new ShuffleReadMetrics + metrics.shuffleFinishTime = (json \ "Shuffle Finish Time").extract[Long] + metrics.totalBlocksFetched = (json \ "Total Blocks Fetched").extract[Int] + metrics.remoteBlocksFetched = (json \ "Remote Blocks Fetched").extract[Int] + metrics.localBlocksFetched = (json \ "Local Blocks Fetched").extract[Int] + metrics.fetchWaitTime = (json \ "Fetch Wait Time").extract[Long] + metrics.remoteBytesRead = (json \ "Remote Bytes Read").extract[Long] + metrics + } + + def shuffleWriteMetricsFromJson(json: JValue): ShuffleWriteMetrics = { + val metrics = new ShuffleWriteMetrics + metrics.shuffleBytesWritten = (json \ "Shuffle Bytes Written").extract[Long] + metrics.shuffleWriteTime = (json \ "Shuffle Write Time").extract[Long] + metrics + } + + def taskEndReasonFromJson(json: JValue): TaskEndReason = { + val success = Utils.getFormattedClassName(Success) + val resubmitted = Utils.getFormattedClassName(Resubmitted) + val fetchFailed = Utils.getFormattedClassName(FetchFailed) + val exceptionFailure = Utils.getFormattedClassName(ExceptionFailure) + val taskResultLost = Utils.getFormattedClassName(TaskResultLost) + val taskKilled = Utils.getFormattedClassName(TaskKilled) + val executorLostFailure = Utils.getFormattedClassName(ExecutorLostFailure) + val unknownReason = Utils.getFormattedClassName(UnknownReason) + + (json \ "Reason").extract[String] match { + case `success` => Success + case `resubmitted` => Resubmitted + case `fetchFailed` => + val blockManagerAddress = blockManagerIdFromJson(json \ "Block Manager Address") + val shuffleId = (json \ "Shuffle ID").extract[Int] + val mapId = (json \ "Map ID").extract[Int] + val reduceId = (json \ "Reduce ID").extract[Int] + new FetchFailed(blockManagerAddress, shuffleId, mapId, reduceId) + case `exceptionFailure` => + val className = (json \ "Class Name").extract[String] + val description = (json \ "Description").extract[String] + val stackTrace = stackTraceFromJson(json \ "Stack Trace") + val metrics = Utils.jsonOption(json \ "Metrics").map(taskMetricsFromJson) + new ExceptionFailure(className, description, stackTrace, metrics) + case `taskResultLost` => TaskResultLost + case `taskKilled` => TaskKilled + case `executorLostFailure` => ExecutorLostFailure + case `unknownReason` => UnknownReason + } + } + + def blockManagerIdFromJson(json: JValue): BlockManagerId = { + val executorId = (json \ "Executor ID").extract[String] + val host = (json \ "Host").extract[String] + val port = (json \ "Port").extract[Int] + val nettyPort = (json \ "Netty Port").extract[Int] + BlockManagerId(executorId, host, port, nettyPort) + } + + def jobResultFromJson(json: JValue): JobResult = { + val jobSucceeded = Utils.getFormattedClassName(JobSucceeded) + val jobFailed = Utils.getFormattedClassName(JobFailed) + + (json \ "Result").extract[String] match { + case `jobSucceeded` => JobSucceeded + case `jobFailed` => + val exception = exceptionFromJson(json \ "Exception") + val failedStageId = (json \ "Failed Stage ID").extract[Int] + new JobFailed(exception, failedStageId) + } + } + + def rddInfoFromJson(json: JValue): RDDInfo = { + val rddId = (json \ "RDD ID").extract[Int] + val name = (json \ "Name").extract[String] + val storageLevel = storageLevelFromJson(json \ "Storage Level") + val numPartitions = (json \ "Number of Partitions").extract[Int] + val numCachedPartitions = (json \ "Number of Cached Partitions").extract[Int] + val memSize = (json \ "Memory Size").extract[Long] + val diskSize = (json \ "Disk Size").extract[Long] + + val rddInfo = new RDDInfo(rddId, name, numPartitions, storageLevel) + rddInfo.numCachedPartitions = numCachedPartitions + rddInfo.memSize = memSize + rddInfo.diskSize = diskSize + rddInfo + } + + def storageLevelFromJson(json: JValue): StorageLevel = { + val useDisk = (json \ "Use Disk").extract[Boolean] + val useMemory = (json \ "Use Memory").extract[Boolean] + val deserialized = (json \ "Deserialized").extract[Boolean] + val replication = (json \ "Replication").extract[Int] + StorageLevel(useDisk, useMemory, deserialized, replication) + } + + def blockIdFromJson(json: JValue): BlockId = { + val rddBlockId = Utils.getFormattedClassName(RDDBlockId) + val shuffleBlockId = Utils.getFormattedClassName(ShuffleBlockId) + val broadcastBlockId = Utils.getFormattedClassName(BroadcastBlockId) + val broadcastHelperBlockId = Utils.getFormattedClassName(BroadcastHelperBlockId) + val taskResultBlockId = Utils.getFormattedClassName(TaskResultBlockId) + val streamBlockId = Utils.getFormattedClassName(StreamBlockId) + val tempBlockId = Utils.getFormattedClassName(TempBlockId) + val testBlockId = Utils.getFormattedClassName(TestBlockId) + + (json \ "Type").extract[String] match { + case `rddBlockId` => + val rddId = (json \ "RDD ID").extract[Int] + val splitIndex = (json \ "Split Index").extract[Int] + new RDDBlockId(rddId, splitIndex) + case `shuffleBlockId` => + val shuffleId = (json \ "Shuffle ID").extract[Int] + val mapId = (json \ "Map ID").extract[Int] + val reduceId = (json \ "Reduce ID").extract[Int] + new ShuffleBlockId(shuffleId, mapId, reduceId) + case `broadcastBlockId` => + val broadcastId = (json \ "Broadcast ID").extract[Long] + new BroadcastBlockId(broadcastId) + case `broadcastHelperBlockId` => + val broadcastBlockId = + blockIdFromJson(json \ "Broadcast Block ID").asInstanceOf[BroadcastBlockId] + val hType = (json \ "Helper Type").extract[String] + new BroadcastHelperBlockId(broadcastBlockId, hType) + case `taskResultBlockId` => + val taskId = (json \ "Task ID").extract[Long] + new TaskResultBlockId(taskId) + case `streamBlockId` => + val streamId = (json \ "Stream ID").extract[Int] + val uniqueId = (json \ "Unique ID").extract[Long] + new StreamBlockId(streamId, uniqueId) + case `tempBlockId` => + val tempId = UUIDFromJson(json \ "Temp ID") + new TempBlockId(tempId) + case `testBlockId` => + val testId = (json \ "Test ID").extract[String] + new TestBlockId(testId) + } + } + + def blockStatusFromJson(json: JValue): BlockStatus = { + val storageLevel = storageLevelFromJson(json \ "Storage Level") + val memorySize = (json \ "Memory Size").extract[Long] + val diskSize = (json \ "Disk Size").extract[Long] + BlockStatus(storageLevel, memorySize, diskSize) + } + + + /** -------------------------------- * + * Util JSON deserialization methods | + * --------------------------------- */ + + def mapFromJson(json: JValue): Map[String, String] = { + val jsonFields = json.asInstanceOf[JObject].obj + jsonFields.map { case JField(k, JString(v)) => (k, v) }.toMap + } + + def propertiesFromJson(json: JValue): Properties = { + val properties = new Properties() + if (json != JNothing) { + mapFromJson(json).map { case (k, v) => properties.setProperty(k, v) } + } + properties + } + + def UUIDFromJson(json: JValue): UUID = { + val leastSignificantBits = (json \ "Least Significant Bits").extract[Long] + val mostSignificantBits = (json \ "Most Significant Bits").extract[Long] + new UUID(leastSignificantBits, mostSignificantBits) + } + + def stackTraceFromJson(json: JValue): Array[StackTraceElement] = { + json.extract[List[JValue]].map { line => + val declaringClass = (line \ "Declaring Class").extract[String] + val methodName = (line \ "Method Name").extract[String] + val fileName = (line \ "File Name").extract[String] + val lineNumber = (line \ "Line Number").extract[Int] + new StackTraceElement(declaringClass, methodName, fileName, lineNumber) + }.toArray + } + + def exceptionFromJson(json: JValue): Exception = { + val e = new Exception((json \ "Message").extract[String]) + e.setStackTrace(stackTraceFromJson(json \ "Stack Trace")) + e + } + +} diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 38a275d438959..13d9dbdd9af2d 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -32,11 +32,11 @@ import scala.reflect.ClassTag import com.google.common.io.Files import com.google.common.util.concurrent.ThreadFactoryBuilder import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} +import org.json4s._ import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} -import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} import org.apache.spark.deploy.SparkHadoopUtil - +import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} /** * Various utility methods used by Spark. @@ -245,7 +245,7 @@ private[spark] object Utils extends Logging { val userCred = securityMgr.getSecretKey() if (userCred == null) throw new Exception("Secret key is null with authentication on") val userInfo = securityMgr.getHttpUser() + ":" + userCred - new URI(uri.getScheme(), userInfo, uri.getHost(), uri.getPort(), uri.getPath(), + new URI(uri.getScheme(), userInfo, uri.getHost(), uri.getPort(), uri.getPath(), uri.getQuery(), uri.getFragment()) } @@ -282,7 +282,7 @@ private[spark] object Utils extends Logging { uc.setConnectTimeout(timeout) uc.setReadTimeout(timeout) uc.connect() - val in = uc.getInputStream(); + val in = uc.getInputStream() val out = new FileOutputStream(tempFile) Utils.copyStream(in, out, true) if (targetFile.exists && !Files.equal(tempFile, targetFile)) { @@ -328,8 +328,7 @@ private[spark] object Utils extends Logging { } case _ => // Use the Hadoop filesystem library, which supports file://, hdfs://, s3://, and others - val conf = SparkHadoopUtil.get.newConfiguration() - val fs = FileSystem.get(uri, conf) + val fs = getHadoopFileSystem(uri) val in = fs.open(new Path(uri)) val out = new FileOutputStream(tempFile) Utils.copyStream(in, out, true) @@ -500,7 +499,7 @@ private[spark] object Utils extends Logging { * millisecond. */ def getUsedTimeMs(startTimeMs: Long): String = { - return " " + (System.currentTimeMillis - startTimeMs) + " ms" + " " + (System.currentTimeMillis - startTimeMs) + " ms" } /** @@ -789,7 +788,7 @@ private[spark] object Utils extends Logging { } var i = 0 while (i < s.length) { - var nextChar = s.charAt(i) + val nextChar = s.charAt(i) if (inDoubleQuote) { if (nextChar == '"') { inDoubleQuote = false @@ -895,4 +894,27 @@ private[spark] object Utils extends Logging { } count } + + /** Return the class name of the given object, removing all dollar signs */ + def getFormattedClassName(obj: AnyRef) = { + obj.getClass.getSimpleName.replace("$", "") + } + + /** Return an option that translates JNothing to None */ + def jsonOption(json: JValue): Option[JValue] = { + json match { + case JNothing => None + case value: JValue => Some(value) + } + } + + /** Return an empty JSON object */ + def emptyJson = JObject(List[JField]()) + + /** + * Return a Hadoop FileSystem with the scheme encoded in the given path. + */ + def getHadoopFileSystem(path: URI): FileSystem = { + FileSystem.get(path, SparkHadoopUtil.get.newConfiguration()) + } } diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 40e853c39ca99..c6b65c7348ae0 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -43,6 +43,7 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.*; +import org.apache.spark.executor.TaskMetrics; import org.apache.spark.partial.BoundedDouble; import org.apache.spark.partial.PartialResult; import org.apache.spark.storage.StorageLevel; @@ -402,16 +403,16 @@ public Boolean call(Double x) { @Test public void javaDoubleRDDHistoGram() { - JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); - // Test using generated buckets - Tuple2 results = rdd.histogram(2); - double[] expected_buckets = {1.0, 2.5, 4.0}; - long[] expected_counts = {2, 2}; - Assert.assertArrayEquals(expected_buckets, results._1, 0.1); - Assert.assertArrayEquals(expected_counts, results._2); - // Test with provided buckets - long[] histogram = rdd.histogram(expected_buckets); - Assert.assertArrayEquals(expected_counts, histogram); + JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); + // Test using generated buckets + Tuple2 results = rdd.histogram(2); + double[] expected_buckets = {1.0, 2.5, 4.0}; + long[] expected_counts = {2, 2}; + Assert.assertArrayEquals(expected_buckets, results._1, 0.1); + Assert.assertArrayEquals(expected_counts, results._2); + // Test with provided buckets + long[] histogram = rdd.histogram(expected_buckets); + Assert.assertArrayEquals(expected_counts, histogram); } @Test @@ -570,7 +571,7 @@ public void persist() { @Test public void iterator() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 2); - TaskContext context = new TaskContext(0, 0, 0, false, false, null); + TaskContext context = new TaskContext(0, 0, 0, false, false, new TaskMetrics()); Assert.assertEquals(1, rdd.iterator(rdd.splits().get(0), context).next().intValue()); } diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala index ea936e815b24a..b86923f07f793 100644 --- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala @@ -23,7 +23,8 @@ import org.scalatest.{BeforeAndAfter, FunSuite} import org.scalatest.mock.EasyMockSugar import org.apache.spark.rdd.RDD -import org.apache.spark.storage.{BlockManager, RDDBlockId, StorageLevel} +import org.apache.spark.executor.TaskMetrics +import org.apache.spark.storage._ // TODO: Test the CacheManager's thread-safety aspects class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar { @@ -54,12 +55,12 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar expecting { blockManager.get(RDDBlockId(0, 0)).andReturn(None) blockManager.put(RDDBlockId(0, 0), ArrayBuffer[Any](1, 2, 3, 4), StorageLevel.MEMORY_ONLY, - true).andReturn(0) + true).andStubReturn(Seq[(BlockId, BlockStatus)]()) } whenExecuting(blockManager) { val context = new TaskContext(0, 0, 0, interrupted = false, runningLocally = false, - taskMetrics = null) + taskMetrics = TaskMetrics.empty()) val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) assert(value.toList === List(1, 2, 3, 4)) } @@ -72,7 +73,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar whenExecuting(blockManager) { val context = new TaskContext(0, 0, 0, interrupted = false, runningLocally = false, - taskMetrics = null) + taskMetrics = TaskMetrics.empty()) val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) assert(value.toList === List(5, 6, 7)) } @@ -86,7 +87,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar whenExecuting(blockManager) { val context = new TaskContext(0, 0, 0, runningLocally = true, interrupted = false, - taskMetrics = null) + taskMetrics = TaskMetrics.empty()) val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) assert(value.toList === List(1, 2, 3, 4)) } diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala index 20c503d30c801..7a39d1af9e2d5 100644 --- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala +++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala @@ -28,7 +28,7 @@ import org.scalatest.{BeforeAndAfter, FunSuite} import org.scalatest.matchers.ShouldMatchers import org.apache.spark.SparkContext._ -import org.apache.spark.scheduler.{SparkListenerTaskStart, SparkListener} +import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskStart} /** * Test suite for cancelling running jobs. We run the cancellation tasks for single job action @@ -89,7 +89,7 @@ class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAf // Add a listener to release the semaphore once any tasks are launched. val sem = new Semaphore(0) - sc.dagScheduler.addSparkListener(new SparkListener { + sc.addSparkListener(new SparkListener { override def onTaskStart(taskStart: SparkListenerTaskStart) { sem.release() } @@ -161,7 +161,7 @@ class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAf { // Add a listener to release the semaphore once any tasks are launched. val sem = new Semaphore(0) - sc.dagScheduler.addSparkListener(new SparkListener { + sc.addSparkListener(new SparkListener { override def onTaskStart(taskStart: SparkListenerTaskStart) { sem.release() } @@ -191,7 +191,7 @@ class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAf { // Add a listener to release the semaphore once any tasks are launched. val sem = new Semaphore(0) - sc.dagScheduler.addSparkListener(new SparkListener { + sc.addSparkListener(new SparkListener { override def onTaskStart(taskStart: SparkListenerTaskStart) { sem.release() } diff --git a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala index 0bac78d8a6bdf..6e7fd55fa4bb1 100644 --- a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala @@ -27,8 +27,11 @@ import org.apache.hadoop.fs.Path import scala.collection.Map import scala.sys.process._ import scala.util.Try + import org.apache.hadoop.io.{Text, LongWritable} +import org.apache.spark.executor.TaskMetrics + class PipedRDDSuite extends FunSuite with SharedSparkContext { test("basic pipe") { @@ -151,7 +154,7 @@ class PipedRDDSuite extends FunSuite with SharedSparkContext { val hadoopPart1 = generateFakeHadoopPartition() val pipedRdd = new PipedRDD(nums, "printenv " + varName) val tContext = new TaskContext(0, 0, 0, interrupted = false, runningLocally = false, - taskMetrics = null) + taskMetrics = TaskMetrics.empty()) val rddIter = pipedRdd.compute(hadoopPart1, tContext) val arr = rddIter.toArray assert(arr(0) == "/some/path") diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index 3bb936790d506..b543471a5d35b 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark import org.scalatest.{FunSuite, PrivateMethodTester} -import org.apache.spark.scheduler.{TaskSchedulerImpl, TaskScheduler} +import org.apache.spark.scheduler.{TaskScheduler, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.{SimrSchedulerBackend, SparkDeploySchedulerBackend} import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} import org.apache.spark.scheduler.local.LocalBackend @@ -32,7 +32,7 @@ class SparkContextSchedulerCreationSuite // real schedulers, so we don't want to create a full SparkContext with the desired scheduler. sc = new SparkContext("local", "test") val createTaskSchedulerMethod = PrivateMethod[TaskScheduler]('createTaskScheduler) - val sched = SparkContext invokePrivate createTaskSchedulerMethod(sc, master, "test") + val sched = SparkContext invokePrivate createTaskSchedulerMethod(sc, master) sched.asInstanceOf[TaskSchedulerImpl] } diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index bae3b37e267d5..9f2924c23b73c 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -20,12 +20,9 @@ package org.apache.spark.deploy import java.io.File import java.util.Date +import com.fasterxml.jackson.core.JsonParseException import org.json4s._ - -import org.json4s.JValue import org.json4s.jackson.JsonMethods -import com.fasterxml.jackson.core.JsonParseException - import org.scalatest.FunSuite import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse} @@ -96,7 +93,7 @@ class JsonProtocolSuite extends FunSuite { def createAppInfo() : ApplicationInfo = { val appInfo = new ApplicationInfo(JsonConstants.appInfoStartTime, - "id", createAppDesc(), JsonConstants.submitDate, null, "appUriStr", Int.MaxValue) + "id", createAppDesc(), JsonConstants.submitDate, null, Int.MaxValue) appInfo.endTime = JsonConstants.currTimeInMillis appInfo } @@ -148,12 +145,12 @@ object JsonConstants { val submitDate = new Date(123456789) val appInfoJsonStr = """ - |{"starttime":3,"id":"id","name":"name","appuiurl":"appUriStr", + |{"starttime":3,"id":"id","name":"name", |"cores":4,"user":"%s", |"memoryperslave":1234,"submitdate":"%s", |"state":"WAITING","duration":%d} """.format(System.getProperty("user.name", ""), - submitDate.toString, (currTimeInMillis - appInfoStartTime)).stripMargin + submitDate.toString, currTimeInMillis - appInfoStartTime).stripMargin val workerInfoJsonStr = """ 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 ad890b4e4d08e..c97543f57d8f3 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -94,7 +94,12 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont cacheLocations.clear() results.clear() mapOutputTracker = new MapOutputTrackerMaster(conf) - scheduler = new DAGScheduler(taskScheduler, mapOutputTracker, blockManagerMaster, sc.env) { + scheduler = new DAGScheduler( + taskScheduler, + sc.listenerBus, + mapOutputTracker, + blockManagerMaster, + sc.env) { override def runLocally(job: ActiveJob) { // don't bother with the thread while unit testing runLocallyWithinThread(job) @@ -422,15 +427,15 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont private def assertDataStructuresEmpty = { assert(scheduler.pendingTasks.isEmpty) assert(scheduler.activeJobs.isEmpty) - assert(scheduler.failed.isEmpty) - assert(scheduler.idToActiveJob.isEmpty) + assert(scheduler.failedStages.isEmpty) + assert(scheduler.stageIdToActiveJob.isEmpty) assert(scheduler.jobIdToStageIds.isEmpty) assert(scheduler.stageIdToJobIds.isEmpty) assert(scheduler.stageIdToStage.isEmpty) assert(scheduler.stageToInfos.isEmpty) assert(scheduler.resultStageToJob.isEmpty) - assert(scheduler.running.isEmpty) + assert(scheduler.runningStages.isEmpty) assert(scheduler.shuffleToMapStage.isEmpty) - assert(scheduler.waiting.isEmpty) + assert(scheduler.waitingStages.isEmpty) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala deleted file mode 100644 index 25fe63c2655a8..0000000000000 --- a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala +++ /dev/null @@ -1,126 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.scheduler - -import org.scalatest.FunSuite -import org.scalatest.matchers.ShouldMatchers - -import org.apache.spark._ -import org.apache.spark.SparkContext._ -import org.apache.spark.rdd.RDD - -class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers { - val WAIT_TIMEOUT_MILLIS = 10000 - - test("inner method") { - sc = new SparkContext("local", "joblogger") - val joblogger = new JobLogger { - def createLogWriterTest(jobID: Int) = createLogWriter(jobID) - def closeLogWriterTest(jobID: Int) = closeLogWriter(jobID) - def getRddNameTest(rdd: RDD[_]) = getRddName(rdd) - def buildJobDepTest(jobID: Int, stage: Stage) = buildJobDep(jobID, stage) - } - type MyRDD = RDD[(Int, Int)] - def makeRdd(numPartitions: Int, dependencies: List[Dependency[_]]): MyRDD = { - val maxPartition = numPartitions - 1 - new MyRDD(sc, dependencies) { - override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = - throw new RuntimeException("should not be reached") - override def getPartitions = (0 to maxPartition).map(i => new Partition { - override def index = i - }).toArray - } - } - val jobID = 5 - val parentRdd = makeRdd(4, Nil) - val shuffleDep = new ShuffleDependency(parentRdd, null) - val rootRdd = makeRdd(4, List(shuffleDep)) - val shuffleMapStage = - new Stage(1, parentRdd, parentRdd.partitions.size, Some(shuffleDep), Nil, jobID, None) - val rootStage = - new Stage(0, rootRdd, rootRdd.partitions.size, None, List(shuffleMapStage), jobID, None) - val rootStageInfo = new StageInfo(rootStage) - - joblogger.onStageSubmitted(SparkListenerStageSubmitted(rootStageInfo, null)) - joblogger.getRddNameTest(parentRdd) should be (parentRdd.getClass.getSimpleName) - parentRdd.setName("MyRDD") - joblogger.getRddNameTest(parentRdd) should be ("MyRDD") - joblogger.createLogWriterTest(jobID) - joblogger.getJobIDtoPrintWriter.size should be (1) - joblogger.buildJobDepTest(jobID, rootStage) - joblogger.getJobIDToStages.get(jobID).get.size should be (2) - joblogger.getStageIDToJobID.get(0) should be (Some(jobID)) - joblogger.getStageIDToJobID.get(1) should be (Some(jobID)) - joblogger.closeLogWriterTest(jobID) - joblogger.getStageIDToJobID.size should be (0) - joblogger.getJobIDToStages.size should be (0) - joblogger.getJobIDtoPrintWriter.size should be (0) - } - - test("inner variables") { - sc = new SparkContext("local[4]", "joblogger") - val joblogger = new JobLogger { - override protected def closeLogWriter(jobID: Int) = - getJobIDtoPrintWriter.get(jobID).foreach { fileWriter => - fileWriter.close() - } - } - sc.addSparkListener(joblogger) - val rdd = sc.parallelize(1 to 1e2.toInt, 4).map{ i => (i % 12, 2 * i) } - rdd.reduceByKey(_+_).collect() - - assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) - - val user = System.getProperty("user.name", SparkContext.SPARK_UNKNOWN_USER) - - joblogger.getLogDir should be ("/tmp/spark-%s".format(user)) - joblogger.getJobIDtoPrintWriter.size should be (1) - joblogger.getStageIDToJobID.size should be (2) - joblogger.getStageIDToJobID.get(0) should be (Some(0)) - joblogger.getStageIDToJobID.get(1) should be (Some(0)) - joblogger.getJobIDToStages.size should be (1) - } - - - test("interface functions") { - sc = new SparkContext("local[4]", "joblogger") - val joblogger = new JobLogger { - var onTaskEndCount = 0 - var onJobEndCount = 0 - var onJobStartCount = 0 - var onStageCompletedCount = 0 - var onStageSubmittedCount = 0 - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = onTaskEndCount += 1 - override def onJobEnd(jobEnd: SparkListenerJobEnd) = onJobEndCount += 1 - override def onJobStart(jobStart: SparkListenerJobStart) = onJobStartCount += 1 - override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = onStageCompletedCount += 1 - override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = onStageSubmittedCount += 1 - } - sc.addSparkListener(joblogger) - val rdd = sc.parallelize(1 to 1e2.toInt, 4).map{ i => (i % 12, 2 * i) } - rdd.reduceByKey(_+_).collect() - - assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) - - joblogger.onJobStartCount should be (1) - joblogger.onJobEndCount should be (1) - joblogger.onTaskEndCount should be (8) - joblogger.onStageSubmittedCount should be (2) - joblogger.onStageCompletedCount should be (2) - } -} diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index 7c4f2b4361892..a25ce35736146 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -17,13 +17,14 @@ package org.apache.spark.scheduler -import scala.collection.mutable.{Buffer, HashSet} +import scala.collection.mutable import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} import org.scalatest.matchers.ShouldMatchers import org.apache.spark.{LocalSparkContext, SparkContext} import org.apache.spark.SparkContext._ +import org.apache.spark.executor.TaskMetrics class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatchers with BeforeAndAfter with BeforeAndAfterAll { @@ -38,43 +39,76 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc System.clearProperty("spark.akka.frameSize") } + test("basic creation and shutdown of LiveListenerBus") { + val counter = new BasicJobCounter + val bus = new LiveListenerBus + bus.addListener(counter) + + // Listener bus hasn't started yet, so posting events should not increment counter + (1 to 5).foreach { _ => bus.post(SparkListenerJobEnd(0, JobSucceeded)) } + assert(counter.count === 0) + + // Starting listener bus should flush all buffered events (asynchronously, hence the sleep) + bus.start() + Thread.sleep(1000) + assert(counter.count === 5) + + // After listener bus has stopped, posting events should not increment counter + bus.stop() + (1 to 5).foreach { _ => bus.post(SparkListenerJobEnd(0, JobSucceeded)) } + assert(counter.count === 5) + + // Listener bus must not be started twice + intercept[IllegalStateException] { + val bus = new LiveListenerBus + bus.start() + bus.start() + } + + // ... or stopped before starting + intercept[IllegalStateException] { + val bus = new LiveListenerBus + bus.stop() + } + } + test("basic creation of StageInfo") { - val listener = new SaveStageInfo + val listener = new SaveStageAndTaskInfo sc.addSparkListener(listener) val rdd1 = sc.parallelize(1 to 100, 4) - val rdd2 = rdd1.map(x => x.toString) + val rdd2 = rdd1.map(_.toString) rdd2.setName("Target RDD") rdd2.count - assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) listener.stageInfos.size should be {1} - val first = listener.stageInfos.head - first.rddName should be {"Target RDD"} - first.numTasks should be {4} - first.numPartitions should be {4} - first.submissionTime should be ('defined) - first.completionTime should be ('defined) - first.taskInfos.length should be {4} + val (stageInfo, taskInfoMetrics) = listener.stageInfos.head + stageInfo.rddInfo.name should be {"Target RDD"} + stageInfo.numTasks should be {4} + stageInfo.rddInfo.numPartitions should be {4} + stageInfo.submissionTime should be ('defined) + stageInfo.completionTime should be ('defined) + taskInfoMetrics.length should be {4} } test("StageInfo with fewer tasks than partitions") { - val listener = new SaveStageInfo + val listener = new SaveStageAndTaskInfo sc.addSparkListener(listener) val rdd1 = sc.parallelize(1 to 100, 4) - val rdd2 = rdd1.map(x => x.toString) + val rdd2 = rdd1.map(_.toString) sc.runJob(rdd2, (items: Iterator[String]) => items.size, Seq(0, 1), true) - assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) listener.stageInfos.size should be {1} - val first = listener.stageInfos.head - first.numTasks should be {2} - first.numPartitions should be {4} + val (stageInfo, _) = listener.stageInfos.head + stageInfo.numTasks should be {2} + stageInfo.rddInfo.numPartitions should be {4} } test("local metrics") { - val listener = new SaveStageInfo + val listener = new SaveStageAndTaskInfo sc.addSparkListener(listener) sc.addSparkListener(new StatsReportListener) //just to make sure some of the tasks take a noticeable amount of time @@ -84,45 +118,45 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc i } - val d = sc.parallelize(0 to 1e4.toInt, 64).map{i => w(i)} + val d = sc.parallelize(0 to 1e4.toInt, 64).map(w) d.count() - assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) listener.stageInfos.size should be (1) - val d2 = d.map{i => w(i) -> i * 2}.setName("shuffle input 1") - - val d3 = d.map{i => w(i) -> (0 to (i % 5))}.setName("shuffle input 2") - - val d4 = d2.cogroup(d3, 64).map{case(k,(v1,v2)) => w(k) -> (v1.size, v2.size)} + val d2 = d.map { i => w(i) -> i * 2 }.setName("shuffle input 1") + val d3 = d.map { i => w(i) -> (0 to (i % 5)) }.setName("shuffle input 2") + val d4 = d2.cogroup(d3, 64).map { case (k, (v1, v2)) => + w(k) -> (v1.size, v2.size) + } d4.setName("A Cogroup") - d4.collectAsMap() - assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) listener.stageInfos.size should be (4) - listener.stageInfos.foreach { stageInfo => - /* small test, so some tasks might take less than 1 millisecond, but average should be greater - * than 0 ms. */ - checkNonZeroAvg(stageInfo.taskInfos.map{_._1.duration}, stageInfo + " duration") + listener.stageInfos.foreach { case (stageInfo, taskInfoMetrics) => + /** + * Small test, so some tasks might take less than 1 millisecond, but average should be greater + * than 0 ms. + */ checkNonZeroAvg( - stageInfo.taskInfos.map{_._2.executorRunTime.toLong}, + taskInfoMetrics.map(_._2.executorRunTime), stageInfo + " executorRunTime") checkNonZeroAvg( - stageInfo.taskInfos.map{_._2.executorDeserializeTime.toLong}, + taskInfoMetrics.map(_._2.executorDeserializeTime), stageInfo + " executorDeserializeTime") - if (stageInfo.rddName == d4.name) { + if (stageInfo.rddInfo.name == d4.name) { checkNonZeroAvg( - stageInfo.taskInfos.map{_._2.shuffleReadMetrics.get.fetchWaitTime}, + taskInfoMetrics.map(_._2.shuffleReadMetrics.get.fetchWaitTime), stageInfo + " fetchWaitTime") } - stageInfo.taskInfos.foreach { case (taskInfo, taskMetrics) => + taskInfoMetrics.foreach { case (taskInfo, taskMetrics) => taskMetrics.resultSize should be > (0l) - if (stageInfo.rddName == d2.name || stageInfo.rddName == d3.name) { + if (stageInfo.rddInfo.name == d2.name || stageInfo.rddInfo.name == d3.name) { taskMetrics.shuffleWriteMetrics should be ('defined) taskMetrics.shuffleWriteMetrics.get.shuffleBytesWritten should be > (0l) } - if (stageInfo.rddName == d4.name) { + if (stageInfo.rddInfo.name == d4.name) { taskMetrics.shuffleReadMetrics should be ('defined) val sm = taskMetrics.shuffleReadMetrics.get sm.totalBlocksFetched should be > (0) @@ -142,10 +176,12 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc System.setProperty("spark.akka.frameSize", "1") val akkaFrameSize = sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.maximum-frame-size").toInt - val result = sc.parallelize(Seq(1), 1).map(x => 1.to(akkaFrameSize).toArray).reduce((x,y) => x) + val result = sc.parallelize(Seq(1), 1) + .map { x => 1.to(akkaFrameSize).toArray } + .reduce { case (x, y) => x } assert(result === 1.to(akkaFrameSize).toArray) - assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) val TASK_INDEX = 0 assert(listener.startedTasks.contains(TASK_INDEX)) assert(listener.startedGettingResultTasks.contains(TASK_INDEX)) @@ -157,13 +193,13 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc sc.addSparkListener(listener) // Make a task whose result is larger than the akka frame size - val result = sc.parallelize(Seq(1), 1).map(x => 2 * x).reduce((x, y) => x) + val result = sc.parallelize(Seq(1), 1).map(2 * _).reduce { case (x, y) => x } assert(result === 2) - assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) val TASK_INDEX = 0 assert(listener.startedTasks.contains(TASK_INDEX)) - assert(listener.startedGettingResultTasks.isEmpty == true) + assert(listener.startedGettingResultTasks.isEmpty) assert(listener.endedTasks.contains(TASK_INDEX)) } @@ -204,17 +240,33 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc assert(m.sum / m.size.toDouble > 0.0, msg) } - class SaveStageInfo extends SparkListener { - val stageInfos = Buffer[StageInfo]() + class BasicJobCounter extends SparkListener { + var count = 0 + override def onJobEnd(job: SparkListenerJobEnd) = count += 1 + } + + class SaveStageAndTaskInfo extends SparkListener { + val stageInfos = mutable.Map[StageInfo, Seq[(TaskInfo, TaskMetrics)]]() + var taskInfoMetrics = mutable.Buffer[(TaskInfo, TaskMetrics)]() + + override def onTaskEnd(task: SparkListenerTaskEnd) { + val info = task.taskInfo + val metrics = task.taskMetrics + if (info != null && metrics != null) { + taskInfoMetrics += ((info, metrics)) + } + } + override def onStageCompleted(stage: SparkListenerStageCompleted) { - stageInfos += stage.stage + stageInfos(stage.stageInfo) = taskInfoMetrics + taskInfoMetrics = mutable.Buffer.empty } } class SaveTaskEvents extends SparkListener { - val startedTasks = new HashSet[Int]() - val startedGettingResultTasks = new HashSet[Int]() - val endedTasks = new HashSet[Int]() + val startedTasks = new mutable.HashSet[Int]() + val startedGettingResultTasks = new mutable.HashSet[Int]() + val endedTasks = new mutable.HashSet[Int]() override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { startedTasks += taskStart.taskInfo.index diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index 6b0800af9c6d0..9274e01632d58 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -19,8 +19,6 @@ package org.apache.spark.scheduler import java.util.Properties -import scala.collection.mutable.ArrayBuffer - import org.scalatest.FunSuite import org.apache.spark._ @@ -270,9 +268,9 @@ class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Loggin val taskScheduler = new TaskSchedulerImpl(sc) taskScheduler.initialize(new FakeSchedulerBackend) // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. - var dagScheduler = new DAGScheduler(taskScheduler) { + val dagScheduler = new DAGScheduler(sc, taskScheduler) { override def taskStarted(task: Task[_], taskInfo: TaskInfo) {} - override def executorGained(execId: String, host: String) {} + override def executorAdded(execId: String, host: String) {} } val numFreeCores = 1 @@ -291,7 +289,7 @@ class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Loggin assert(1 === taskDescriptions.length) taskDescriptions(0).executorId } - var count = selectedExecutorIds.count(_ == workerOffers(0).executorId) + val count = selectedExecutorIds.count(_ == workerOffers(0).executorId) assert(count > 0) assert(count < numTrials) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 73153d23c4698..9af5d3a303ae2 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -26,7 +26,9 @@ import org.apache.spark._ import org.apache.spark.executor.TaskMetrics import org.apache.spark.util.FakeClock -class FakeDAGScheduler(taskScheduler: FakeTaskScheduler) extends DAGScheduler(taskScheduler) { +class FakeDAGScheduler(sc: SparkContext, taskScheduler: FakeTaskScheduler) + extends DAGScheduler(sc) { + override def taskStarted(task: Task[_], taskInfo: TaskInfo) { taskScheduler.startedTasks += taskInfo.index } @@ -41,7 +43,7 @@ class FakeDAGScheduler(taskScheduler: FakeTaskScheduler) extends DAGScheduler(ta taskScheduler.endedTasks(taskInfo.index) = reason } - override def executorGained(execId: String, host: String) {} + override def executorAdded(execId: String, host: String) {} override def executorLost(execId: String) {} @@ -66,7 +68,7 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex val executors = new mutable.HashMap[String, String] ++ liveExecutors - dagScheduler = new FakeDAGScheduler(this) + dagScheduler = new FakeDAGScheduler(sc, this) def removeExecutor(execId: String): Unit = executors -= execId diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 1036b9f34e9dd..e83cd55e73691 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -28,7 +28,8 @@ import org.scalatest.concurrent.Timeouts._ import org.scalatest.matchers.ShouldMatchers._ import org.scalatest.time.SpanSugar._ -import org.apache.spark.{SecurityManager, SparkConf, SparkContext} +import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} import org.apache.spark.util.{AkkaUtils, ByteBufferInputStream, SizeEstimator, Utils} @@ -57,7 +58,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT conf.set("spark.driver.port", boundPort.toString) master = new BlockManagerMaster( - actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf))), conf) + actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf, new LiveListenerBus))), + conf) // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case oldArch = System.setProperty("os.arch", "amd64") @@ -492,12 +494,9 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY_SER) store.putSingle("a3", a3, StorageLevel.DISK_ONLY) // At this point LRU should not kick in because a3 is only on disk - assert(store.getSingle("a1").isDefined, "a2 was not in store") - assert(store.getSingle("a2").isDefined, "a3 was not in store") - assert(store.getSingle("a3").isDefined, "a1 was not in store") - assert(store.getSingle("a1").isDefined, "a2 was not in store") - assert(store.getSingle("a2").isDefined, "a3 was not in store") - assert(store.getSingle("a3").isDefined, "a1 was not in store") + assert(store.getSingle("a1").isDefined, "a1 was not in store") + assert(store.getSingle("a2").isDefined, "a2 was not in store") + assert(store.getSingle("a3").isDefined, "a3 was not in store") // Now let's add in a4, which uses both disk and memory; a1 should drop out store.putSingle("a4", a4, StorageLevel.MEMORY_AND_DISK_SER) assert(store.getSingle("a1") == None, "a1 was in store") @@ -663,6 +662,60 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } } + test("updated block statuses") { + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + val list = List.fill(2)(new Array[Byte](200)) + val bigList = List.fill(8)(new Array[Byte](200)) + + // 1 updated block (i.e. list1) + val updatedBlocks1 = + store.put("list1", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + assert(updatedBlocks1.size === 1) + assert(updatedBlocks1.head._1 === TestBlockId("list1")) + assert(updatedBlocks1.head._2.storageLevel === StorageLevel.MEMORY_ONLY) + + // 1 updated block (i.e. list2) + val updatedBlocks2 = + store.put("list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + assert(updatedBlocks2.size === 1) + assert(updatedBlocks2.head._1 === TestBlockId("list2")) + assert(updatedBlocks2.head._2.storageLevel === StorageLevel.MEMORY_ONLY) + + // 2 updated blocks - list1 is kicked out of memory while list3 is added + val updatedBlocks3 = + store.put("list3", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + assert(updatedBlocks3.size === 2) + updatedBlocks3.foreach { case (id, status) => + id match { + case TestBlockId("list1") => assert(status.storageLevel === StorageLevel.NONE) + case TestBlockId("list3") => assert(status.storageLevel === StorageLevel.MEMORY_ONLY) + case _ => fail("Updated block is neither list1 nor list3") + } + } + assert(store.get("list3").isDefined, "list3 was not in store") + + // 2 updated blocks - list2 is kicked out of memory (but put on disk) while list4 is added + val updatedBlocks4 = + store.put("list4", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + assert(updatedBlocks4.size === 2) + updatedBlocks4.foreach { case (id, status) => + id match { + case TestBlockId("list2") => assert(status.storageLevel === StorageLevel.DISK_ONLY) + case TestBlockId("list4") => assert(status.storageLevel === StorageLevel.MEMORY_ONLY) + case _ => fail("Updated block is neither list2 nor list4") + } + } + assert(store.get("list4").isDefined, "list4 was not in store") + + // No updated blocks - nothing is kicked out of memory because list5 is too big to be added + val updatedBlocks5 = + store.put("list5", bigList.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + assert(updatedBlocks5.size === 0) + assert(store.get("list2").isDefined, "list2 was not in store") + assert(store.get("list4").isDefined, "list4 was not in store") + assert(!store.get("list5").isDefined, "list5 was in store") + } + test("SPARK-1194 regression: fix the same-RDD rule for cache replacement") { store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) store.putSingle(rdd(0, 0), new Array[Byte](400), StorageLevel.MEMORY_ONLY) diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index 30415814adbba..45c322427930d 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -22,6 +22,7 @@ import java.net.ServerSocket import scala.util.{Failure, Success, Try} import org.eclipse.jetty.server.Server +import org.eclipse.jetty.servlet.ServletContextHandler import org.scalatest.FunSuite import org.apache.spark.SparkConf @@ -36,22 +37,27 @@ class UISuite extends FunSuite { case Failure(e) => // Either case server port is busy hence setup for test complete } - val (jettyServer1, boundPort1) = JettyUtils.startJettyServer("0.0.0.0", startPort, Seq(), - new SparkConf) - val (jettyServer2, boundPort2) = JettyUtils.startJettyServer("0.0.0.0", startPort, Seq(), - new SparkConf) + val serverInfo1 = JettyUtils.startJettyServer( + "0.0.0.0", startPort, Seq[ServletContextHandler](), new SparkConf) + val serverInfo2 = JettyUtils.startJettyServer( + "0.0.0.0", startPort, Seq[ServletContextHandler](), new SparkConf) // Allow some wiggle room in case ports on the machine are under contention + val boundPort1 = serverInfo1.boundPort + val boundPort2 = serverInfo2.boundPort assert(boundPort1 > startPort && boundPort1 < startPort + 10) assert(boundPort2 > boundPort1 && boundPort2 < boundPort1 + 10) } test("jetty binds to port 0 correctly") { - val (jettyServer, boundPort) = JettyUtils.startJettyServer("0.0.0.0", 0, Seq(), new SparkConf) - assert(jettyServer.getState === "STARTED") + val serverInfo = JettyUtils.startJettyServer( + "0.0.0.0", 0, Seq[ServletContextHandler](), new SparkConf) + val server = serverInfo.server + val boundPort = serverInfo.boundPort + assert(server.getState === "STARTED") assert(boundPort != 0) - Try {new ServerSocket(boundPort)} match { + Try { new ServerSocket(boundPort) } match { case Success(s) => fail("Port %s doesn't seem used by jetty server".format(boundPort)) - case Failure (e) => + case Failure(e) => } } } diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index 8ca863e8b3667..d8a3e859f85cd 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -22,11 +22,12 @@ import org.scalatest.FunSuite import org.apache.spark.{LocalSparkContext, SparkContext, Success} import org.apache.spark.executor.{ShuffleReadMetrics, TaskMetrics} import org.apache.spark.scheduler._ +import org.apache.spark.util.Utils class JobProgressListenerSuite extends FunSuite with LocalSparkContext { test("test executor id to summary") { val sc = new SparkContext("local", "test") - val listener = new JobProgressListener(sc) + val listener = new JobProgressListener(sc.conf) val taskMetrics = new TaskMetrics() val shuffleReadMetrics = new ShuffleReadMetrics() @@ -38,16 +39,17 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext { taskMetrics.shuffleReadMetrics = Some(shuffleReadMetrics) var taskInfo = new TaskInfo(1234L, 0, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL) taskInfo.finishTime = 1 - listener.onTaskEnd(new SparkListenerTaskEnd( - new ShuffleMapTask(0, null, null, 0, null), Success, taskInfo, taskMetrics)) + var task = new ShuffleMapTask(0, null, null, 0, null) + val taskType = Utils.getFormattedClassName(task) + listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) assert(listener.stageIdToExecutorSummaries.getOrElse(0, fail()).getOrElse("exe-1", fail()) .shuffleRead == 1000) // finish a task with unknown executor-id, nothing should happen taskInfo = new TaskInfo(1234L, 0, 1000L, "exe-unknown", "host1", TaskLocality.NODE_LOCAL) taskInfo.finishTime = 1 - listener.onTaskEnd(new SparkListenerTaskEnd( - new ShuffleMapTask(0, null, null, 0, null), Success, taskInfo, taskMetrics)) + task = new ShuffleMapTask(0, null, null, 0, null) + listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) assert(listener.stageIdToExecutorSummaries.size == 1) // finish this task, should get updated duration @@ -55,8 +57,8 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext { taskMetrics.shuffleReadMetrics = Some(shuffleReadMetrics) taskInfo = new TaskInfo(1235L, 0, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL) taskInfo.finishTime = 1 - listener.onTaskEnd(new SparkListenerTaskEnd( - new ShuffleMapTask(0, null, null, 0, null), Success, taskInfo, taskMetrics)) + task = new ShuffleMapTask(0, null, null, 0, null) + listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) assert(listener.stageIdToExecutorSummaries.getOrElse(0, fail()).getOrElse("exe-1", fail()) .shuffleRead == 2000) @@ -65,8 +67,8 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext { taskMetrics.shuffleReadMetrics = Some(shuffleReadMetrics) taskInfo = new TaskInfo(1236L, 0, 0L, "exe-2", "host1", TaskLocality.NODE_LOCAL) taskInfo.finishTime = 1 - listener.onTaskEnd(new SparkListenerTaskEnd( - new ShuffleMapTask(0, null, null, 0, null), Success, taskInfo, taskMetrics)) + task = new ShuffleMapTask(0, null, null, 0, null) + listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) assert(listener.stageIdToExecutorSummaries.getOrElse(0, fail()).getOrElse("exe-2", fail()) .shuffleRead == 1000) } diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala new file mode 100644 index 0000000000000..67c0a434c9b52 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -0,0 +1,559 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import java.util.{Properties, UUID} + +import scala.collection.Map + +import org.json4s.jackson.JsonMethods._ +import org.scalatest.FunSuite + +import org.apache.spark._ +import org.apache.spark.executor._ +import org.apache.spark.scheduler._ +import org.apache.spark.storage._ + +class JsonProtocolSuite extends FunSuite { + + test("SparkListenerEvent") { + val stageSubmitted = + SparkListenerStageSubmitted(makeStageInfo(100, 200, 300, 400L, 500L), properties) + val stageCompleted = SparkListenerStageCompleted(makeStageInfo(101, 201, 301, 401L, 501L)) + val taskStart = SparkListenerTaskStart(111, makeTaskInfo(222L, 333, 444L)) + val taskGettingResult = SparkListenerTaskGettingResult(makeTaskInfo(1000L, 2000, 3000L)) + val taskEnd = SparkListenerTaskEnd(1, "ShuffleMapTask", Success, + makeTaskInfo(123L, 234, 345L), makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800)) + val jobStart = SparkListenerJobStart(10, Seq[Int](1, 2, 3, 4), properties) + val jobEnd = SparkListenerJobEnd(20, JobSucceeded) + val environmentUpdate = SparkListenerEnvironmentUpdate(Map[String, Seq[(String, String)]]( + "JVM Information" -> Seq(("GC speed", "9999 objects/s"), ("Java home", "Land of coffee")), + "Spark Properties" -> Seq(("Job throughput", "80000 jobs/s, regardless of job type")), + "System Properties" -> Seq(("Username", "guest"), ("Password", "guest")), + "Classpath Entries" -> Seq(("Super library", "/tmp/super_library")) + )) + val blockManagerAdded = SparkListenerBlockManagerAdded( + BlockManagerId("Stars", "In your multitude...", 300, 400), 500) + val blockManagerRemoved = SparkListenerBlockManagerRemoved( + BlockManagerId("Scarce", "to be counted...", 100, 200)) + val unpersistRdd = SparkListenerUnpersistRDD(12345) + + testEvent(stageSubmitted, stageSubmittedJsonString) + testEvent(stageCompleted, stageCompletedJsonString) + testEvent(taskStart, taskStartJsonString) + testEvent(taskGettingResult, taskGettingResultJsonString) + testEvent(taskEnd, taskEndJsonString) + testEvent(jobStart, jobStartJsonString) + testEvent(jobEnd, jobEndJsonString) + testEvent(environmentUpdate, environmentUpdateJsonString) + testEvent(blockManagerAdded, blockManagerAddedJsonString) + testEvent(blockManagerRemoved, blockManagerRemovedJsonString) + testEvent(unpersistRdd, unpersistRDDJsonString) + } + + test("Dependent Classes") { + testRDDInfo(makeRddInfo(2, 3, 4, 5L, 6L)) + testStageInfo(makeStageInfo(10, 20, 30, 40L, 50L)) + testTaskInfo(makeTaskInfo(999L, 888, 777L)) + testTaskMetrics(makeTaskMetrics(33333L, 44444L, 55555L, 66666L, 7, 8)) + testBlockManagerId(BlockManagerId("Hong", "Kong", 500, 1000)) + + // StorageLevel + testStorageLevel(StorageLevel.NONE) + testStorageLevel(StorageLevel.DISK_ONLY) + testStorageLevel(StorageLevel.DISK_ONLY_2) + testStorageLevel(StorageLevel.MEMORY_ONLY) + testStorageLevel(StorageLevel.MEMORY_ONLY_2) + testStorageLevel(StorageLevel.MEMORY_ONLY_SER) + testStorageLevel(StorageLevel.MEMORY_ONLY_SER_2) + testStorageLevel(StorageLevel.MEMORY_AND_DISK) + testStorageLevel(StorageLevel.MEMORY_AND_DISK_2) + testStorageLevel(StorageLevel.MEMORY_AND_DISK_SER) + testStorageLevel(StorageLevel.MEMORY_AND_DISK_SER_2) + + // JobResult + val exception = new Exception("Out of Memory! Please restock film.") + exception.setStackTrace(stackTrace) + val jobFailed = JobFailed(exception, 2) + testJobResult(JobSucceeded) + testJobResult(jobFailed) + + // TaskEndReason + val fetchFailed = FetchFailed(BlockManagerId("With or", "without you", 15, 16), 17, 18, 19) + val exceptionFailure = ExceptionFailure("To be", "or not to be", stackTrace, None) + testTaskEndReason(Success) + testTaskEndReason(Resubmitted) + testTaskEndReason(fetchFailed) + testTaskEndReason(exceptionFailure) + testTaskEndReason(TaskResultLost) + testTaskEndReason(TaskKilled) + testTaskEndReason(ExecutorLostFailure) + testTaskEndReason(UnknownReason) + + // BlockId + testBlockId(RDDBlockId(1, 2)) + testBlockId(ShuffleBlockId(1, 2, 3)) + testBlockId(BroadcastBlockId(1L)) + testBlockId(BroadcastHelperBlockId(BroadcastBlockId(2L), "Spark")) + testBlockId(TaskResultBlockId(1L)) + testBlockId(StreamBlockId(1, 2L)) + testBlockId(TempBlockId(UUID.randomUUID())) + } + + + /** -------------------------- * + | Helper test running methods | + * --------------------------- */ + + private def testEvent(event: SparkListenerEvent, jsonString: String) { + val actualJsonString = compact(render(JsonProtocol.sparkEventToJson(event))) + val newEvent = JsonProtocol.sparkEventFromJson(parse(actualJsonString)) + assertJsonStringEquals(jsonString, actualJsonString) + assertEquals(event, newEvent) + } + + private def testRDDInfo(info: RDDInfo) { + val newInfo = JsonProtocol.rddInfoFromJson(JsonProtocol.rddInfoToJson(info)) + assertEquals(info, newInfo) + } + + private def testStageInfo(info: StageInfo) { + val newInfo = JsonProtocol.stageInfoFromJson(JsonProtocol.stageInfoToJson(info)) + assertEquals(info, newInfo) + } + + private def testStorageLevel(level: StorageLevel) { + val newLevel = JsonProtocol.storageLevelFromJson(JsonProtocol.storageLevelToJson(level)) + assertEquals(level, newLevel) + } + + private def testTaskMetrics(metrics: TaskMetrics) { + val newMetrics = JsonProtocol.taskMetricsFromJson(JsonProtocol.taskMetricsToJson(metrics)) + assertEquals(metrics, newMetrics) + } + + private def testBlockManagerId(id: BlockManagerId) { + val newId = JsonProtocol.blockManagerIdFromJson(JsonProtocol.blockManagerIdToJson(id)) + assertEquals(id, newId) + } + + private def testTaskInfo(info: TaskInfo) { + val newInfo = JsonProtocol.taskInfoFromJson(JsonProtocol.taskInfoToJson(info)) + assertEquals(info, newInfo) + } + + private def testJobResult(result: JobResult) { + val newResult = JsonProtocol.jobResultFromJson(JsonProtocol.jobResultToJson(result)) + assertEquals(result, newResult) + } + + private def testTaskEndReason(reason: TaskEndReason) { + val newReason = JsonProtocol.taskEndReasonFromJson(JsonProtocol.taskEndReasonToJson(reason)) + assertEquals(reason, newReason) + } + + private def testBlockId(blockId: BlockId) { + val newBlockId = JsonProtocol.blockIdFromJson(JsonProtocol.blockIdToJson(blockId)) + blockId == newBlockId + } + + + /** -------------------------------- * + | Util methods for comparing events | + * --------------------------------- */ + + private def assertEquals(event1: SparkListenerEvent, event2: SparkListenerEvent) { + (event1, event2) match { + case (e1: SparkListenerStageSubmitted, e2: SparkListenerStageSubmitted) => + assert(e1.properties == e2.properties) + assertEquals(e1.stageInfo, e2.stageInfo) + case (e1: SparkListenerStageCompleted, e2: SparkListenerStageCompleted) => + assertEquals(e1.stageInfo, e2.stageInfo) + case (e1: SparkListenerTaskStart, e2: SparkListenerTaskStart) => + assert(e1.stageId == e2.stageId) + assertEquals(e1.taskInfo, e2.taskInfo) + case (e1: SparkListenerTaskGettingResult, e2: SparkListenerTaskGettingResult) => + assertEquals(e1.taskInfo, e2.taskInfo) + case (e1: SparkListenerTaskEnd, e2: SparkListenerTaskEnd) => + assert(e1.stageId == e2.stageId) + assert(e1.taskType == e2.taskType) + assertEquals(e1.reason, e2.reason) + assertEquals(e1.taskInfo, e2.taskInfo) + assertEquals(e1.taskMetrics, e2.taskMetrics) + case (e1: SparkListenerJobStart, e2: SparkListenerJobStart) => + assert(e1.jobId == e2.jobId) + assert(e1.properties == e2.properties) + assertSeqEquals(e1.stageIds, e2.stageIds, (i1: Int, i2: Int) => assert(i1 == i2)) + case (e1: SparkListenerJobEnd, e2: SparkListenerJobEnd) => + assert(e1.jobId == e2.jobId) + assertEquals(e1.jobResult, e2.jobResult) + case (e1: SparkListenerEnvironmentUpdate, e2: SparkListenerEnvironmentUpdate) => + assertEquals(e1.environmentDetails, e2.environmentDetails) + case (e1: SparkListenerBlockManagerAdded, e2: SparkListenerBlockManagerAdded) => + assert(e1.maxMem == e2.maxMem) + assertEquals(e1.blockManagerId, e2.blockManagerId) + case (e1: SparkListenerBlockManagerRemoved, e2: SparkListenerBlockManagerRemoved) => + assertEquals(e1.blockManagerId, e2.blockManagerId) + case (e1: SparkListenerUnpersistRDD, e2: SparkListenerUnpersistRDD) => + assert(e1.rddId == e2.rddId) + case (SparkListenerShutdown, SparkListenerShutdown) => + case _ => fail("Events don't match in types!") + } + } + + private def assertEquals(info1: StageInfo, info2: StageInfo) { + assert(info1.stageId == info2.stageId) + assert(info1.name == info2.name) + assert(info1.numTasks == info2.numTasks) + assert(info1.submissionTime == info2.submissionTime) + assert(info1.completionTime == info2.completionTime) + assert(info1.emittedTaskSizeWarning == info2.emittedTaskSizeWarning) + assertEquals(info1.rddInfo, info2.rddInfo) + } + + private def assertEquals(info1: RDDInfo, info2: RDDInfo) { + assert(info1.id == info2.id) + assert(info1.name == info2.name) + assert(info1.numPartitions == info2.numPartitions) + assert(info1.numCachedPartitions == info2.numCachedPartitions) + assert(info1.memSize == info2.memSize) + assert(info1.diskSize == info2.diskSize) + assertEquals(info1.storageLevel, info2.storageLevel) + } + + private def assertEquals(level1: StorageLevel, level2: StorageLevel) { + assert(level1.useDisk == level2.useDisk) + assert(level1.useMemory == level2.useMemory) + assert(level1.deserialized == level2.deserialized) + assert(level1.replication == level2.replication) + } + + private def assertEquals(info1: TaskInfo, info2: TaskInfo) { + assert(info1.taskId == info2.taskId) + assert(info1.index == info2.index) + assert(info1.launchTime == info2.launchTime) + assert(info1.executorId == info2.executorId) + assert(info1.host == info2.host) + assert(info1.taskLocality == info2.taskLocality) + assert(info1.gettingResultTime == info2.gettingResultTime) + assert(info1.finishTime == info2.finishTime) + assert(info1.failed == info2.failed) + assert(info1.serializedSize == info2.serializedSize) + } + + private def assertEquals(metrics1: TaskMetrics, metrics2: TaskMetrics) { + assert(metrics1.hostname == metrics2.hostname) + assert(metrics1.executorDeserializeTime == metrics2.executorDeserializeTime) + assert(metrics1.resultSize == metrics2.resultSize) + assert(metrics1.jvmGCTime == metrics2.jvmGCTime) + assert(metrics1.resultSerializationTime == metrics2.resultSerializationTime) + assert(metrics1.memoryBytesSpilled == metrics2.memoryBytesSpilled) + assert(metrics1.diskBytesSpilled == metrics2.diskBytesSpilled) + assertOptionEquals( + metrics1.shuffleReadMetrics, metrics2.shuffleReadMetrics, assertShuffleReadEquals) + assertOptionEquals( + metrics1.shuffleWriteMetrics, metrics2.shuffleWriteMetrics, assertShuffleWriteEquals) + assertOptionEquals(metrics1.updatedBlocks, metrics2.updatedBlocks, assertBlocksEquals) + } + + private def assertEquals(metrics1: ShuffleReadMetrics, metrics2: ShuffleReadMetrics) { + assert(metrics1.shuffleFinishTime == metrics2.shuffleFinishTime) + assert(metrics1.totalBlocksFetched == metrics2.totalBlocksFetched) + assert(metrics1.remoteBlocksFetched == metrics2.remoteBlocksFetched) + assert(metrics1.localBlocksFetched == metrics2.localBlocksFetched) + assert(metrics1.fetchWaitTime == metrics2.fetchWaitTime) + assert(metrics1.remoteBytesRead == metrics2.remoteBytesRead) + } + + private def assertEquals(metrics1: ShuffleWriteMetrics, metrics2: ShuffleWriteMetrics) { + assert(metrics1.shuffleBytesWritten == metrics2.shuffleBytesWritten) + assert(metrics1.shuffleWriteTime == metrics2.shuffleWriteTime) + } + + private def assertEquals(bm1: BlockManagerId, bm2: BlockManagerId) { + assert(bm1.executorId == bm2.executorId) + assert(bm1.host == bm2.host) + assert(bm1.port == bm2.port) + assert(bm1.nettyPort == bm2.nettyPort) + } + + private def assertEquals(result1: JobResult, result2: JobResult) { + (result1, result2) match { + case (JobSucceeded, JobSucceeded) => + case (r1: JobFailed, r2: JobFailed) => + assert(r1.failedStageId == r2.failedStageId) + assertEquals(r1.exception, r2.exception) + case _ => fail("Job results don't match in types!") + } + } + + private def assertEquals(reason1: TaskEndReason, reason2: TaskEndReason) { + (reason1, reason2) match { + case (Success, Success) => + case (Resubmitted, Resubmitted) => + case (r1: FetchFailed, r2: FetchFailed) => + assert(r1.shuffleId == r2.shuffleId) + assert(r1.mapId == r2.mapId) + assert(r1.reduceId == r2.reduceId) + assertEquals(r1.bmAddress, r2.bmAddress) + case (r1: ExceptionFailure, r2: ExceptionFailure) => + assert(r1.className == r2.className) + assert(r1.description == r2.description) + assertSeqEquals(r1.stackTrace, r2.stackTrace, assertStackTraceElementEquals) + assertOptionEquals(r1.metrics, r2.metrics, assertTaskMetricsEquals) + case (TaskResultLost, TaskResultLost) => + case (TaskKilled, TaskKilled) => + case (ExecutorLostFailure, ExecutorLostFailure) => + case (UnknownReason, UnknownReason) => + case _ => fail("Task end reasons don't match in types!") + } + } + + private def assertEquals( + details1: Map[String, Seq[(String, String)]], + details2: Map[String, Seq[(String, String)]]) { + details1.zip(details2).foreach { + case ((key1, values1: Seq[(String, String)]), (key2, values2: Seq[(String, String)])) => + assert(key1 == key2) + values1.zip(values2).foreach { case (v1, v2) => assert(v1 == v2) } + } + } + + private def assertEquals(exception1: Exception, exception2: Exception) { + assert(exception1.getMessage == exception2.getMessage) + assertSeqEquals( + exception1.getStackTrace, + exception2.getStackTrace, + assertStackTraceElementEquals) + } + + private def assertJsonStringEquals(json1: String, json2: String) { + val formatJsonString = (json: String) => json.replaceAll("[\\s|]", "") + formatJsonString(json1) == formatJsonString(json2) + } + + private def assertSeqEquals[T](seq1: Seq[T], seq2: Seq[T], assertEquals: (T, T) => Unit) { + assert(seq1.length == seq2.length) + seq1.zip(seq2).foreach { case (t1, t2) => + assertEquals(t1, t2) + } + } + + private def assertOptionEquals[T]( + opt1: Option[T], + opt2: Option[T], + assertEquals: (T, T) => Unit) { + if (opt1.isDefined) { + assert(opt2.isDefined) + assertEquals(opt1.get, opt2.get) + } else { + assert(!opt2.isDefined) + } + } + + /** + * Use different names for methods we pass in to assertSeqEquals or assertOptionEquals + */ + + private def assertShuffleReadEquals(r1: ShuffleReadMetrics, r2: ShuffleReadMetrics) { + assertEquals(r1, r2) + } + + private def assertShuffleWriteEquals(w1: ShuffleWriteMetrics, w2: ShuffleWriteMetrics) { + assertEquals(w1, w2) + } + + private def assertTaskMetricsEquals(t1: TaskMetrics, t2: TaskMetrics) { + assertEquals(t1, t2) + } + + private def assertBlocksEquals( + blocks1: Seq[(BlockId, BlockStatus)], + blocks2: Seq[(BlockId, BlockStatus)]) = { + assertSeqEquals(blocks1, blocks2, assertBlockEquals) + } + + private def assertBlockEquals(b1: (BlockId, BlockStatus), b2: (BlockId, BlockStatus)) { + assert(b1 == b2) + } + + private def assertStackTraceElementEquals(ste1: StackTraceElement, ste2: StackTraceElement) { + assert(ste1 == ste2) + } + + + /** ----------------------------------- * + | Util methods for constructing events | + * ------------------------------------ */ + + private val properties = { + val p = new Properties + p.setProperty("Ukraine", "Kiev") + p.setProperty("Russia", "Moscow") + p.setProperty("France", "Paris") + p.setProperty("Germany", "Berlin") + p + } + + private val stackTrace = { + Array[StackTraceElement]( + new StackTraceElement("Apollo", "Venus", "Mercury", 42), + new StackTraceElement("Afollo", "Vemus", "Mercurry", 420), + new StackTraceElement("Ayollo", "Vesus", "Blackberry", 4200) + ) + } + + private def makeRddInfo(a: Int, b: Int, c: Int, d: Long, e: Long) = { + val r = new RDDInfo(a, "mayor", b, StorageLevel.MEMORY_AND_DISK) + r.numCachedPartitions = c + r.memSize = d + r.diskSize = e + r + } + + private def makeStageInfo(a: Int, b: Int, c: Int, d: Long, e: Long) = { + new StageInfo(a, "greetings", b, makeRddInfo(a, b, c, d, e)) + } + + private def makeTaskInfo(a: Long, b: Int, c: Long) = { + new TaskInfo(a, b, c, "executor", "your kind sir", TaskLocality.NODE_LOCAL) + } + + private def makeTaskMetrics(a: Long, b: Long, c: Long, d: Long, e: Int, f: Int) = { + val t = new TaskMetrics + val sr = new ShuffleReadMetrics + val sw = new ShuffleWriteMetrics + t.hostname = "localhost" + t.executorDeserializeTime = a + t.executorRunTime = b + t.resultSize = c + t.jvmGCTime = d + t.resultSerializationTime = a + b + t.memoryBytesSpilled = a + c + sr.shuffleFinishTime = b + c + sr.totalBlocksFetched = e + f + sr.remoteBytesRead = b + d + sr.localBlocksFetched = e + sr.fetchWaitTime = a + d + sr.remoteBlocksFetched = f + sw.shuffleBytesWritten = a + b + c + sw.shuffleWriteTime = b + c + d + t.shuffleReadMetrics = Some(sr) + t.shuffleWriteMetrics = Some(sw) + // Make at most 6 blocks + t.updatedBlocks = Some((1 to (e % 5 + 1)).map { i => + (RDDBlockId(e % i, f % i), BlockStatus(StorageLevel.MEMORY_AND_DISK_SER_2, a % i, b % i)) + }.toSeq) + t + } + + + /** --------------------------------------- * + | JSON string representation of each event | + * ---------------------------------------- */ + + private val stageSubmittedJsonString = + """ + {"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":100,"Stage Name": + "greetings","Number of Tasks":200,"RDD Info":{"RDD ID":100,"Name":"mayor","Storage + Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1}, + "Number of Partitions":200,"Number of Cached Partitions":300,"Memory Size":400, + "Disk Size":500},"Emitted Task Size Warning":false},"Properties":{"France":"Paris", + "Germany":"Berlin","Russia":"Moscow","Ukraine":"Kiev"}} + """ + + private val stageCompletedJsonString = + """ + {"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":101,"Stage Name": + "greetings","Number of Tasks":201,"RDD Info":{"RDD ID":101,"Name":"mayor","Storage + Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1}, + "Number of Partitions":201,"Number of Cached Partitions":301,"Memory Size":401, + "Disk Size":501},"Emitted Task Size Warning":false}} + """ + + private val taskStartJsonString = + """ + {"Event":"SparkListenerTaskStart","Stage ID":111,"Task Info":{"Task ID":222, + "Index":333,"Launch Time":444,"Executor ID":"executor","Host":"your kind sir", + "Locality":"NODE_LOCAL","Getting Result Time":0,"Finish Time":0,"Failed":false, + "Serialized Size":0}} + """ + + private val taskGettingResultJsonString = + """ + {"Event":"SparkListenerTaskGettingResult","Task Info":{"Task ID":1000,"Index": + 2000,"Launch Time":3000,"Executor ID":"executor","Host":"your kind sir", + "Locality":"NODE_LOCAL","Getting Result Time":0,"Finish Time":0,"Failed":false, + "Serialized Size":0}} + """ + + private val taskEndJsonString = + """ + {"Event":"SparkListenerTaskEnd","Stage ID":1,"Task Type":"ShuffleMapTask", + "Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":123,"Index": + 234,"Launch Time":345,"Executor ID":"executor","Host":"your kind sir", + "Locality":"NODE_LOCAL","Getting Result Time":0,"Finish Time":0,"Failed": + false,"Serialized Size":0},"Task Metrics":{"Host Name":"localhost", + "Executor Deserialize Time":300,"Executor Run Time":400,"Result Size":500, + "JVM GC Time":600,"Result Serialization Time":700,"Memory Bytes Spilled": + 800,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Shuffle Finish Time": + 900,"Total Blocks Fetched":1500,"Remote Blocks Fetched":800,"Local Blocks Fetched": + 700,"Fetch Wait Time":900,"Remote Bytes Read":1000},"Shuffle Write Metrics": + {"Shuffle Bytes Written":1200,"Shuffle Write Time":1500},"Updated Blocks": + [{"Block ID":{"Type":"RDDBlockId","RDD ID":0,"Split Index":0},"Status": + {"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":false, + "Replication":2},"Memory Size":0,"Disk Size":0}}]}} + """ + + private val jobStartJsonString = + """ + {"Event":"SparkListenerJobStart","Job ID":10,"Stage IDs":[1,2,3,4],"Properties": + {"France":"Paris","Germany":"Berlin","Russia":"Moscow","Ukraine":"Kiev"}} + """ + + private val jobEndJsonString = + """ + {"Event":"SparkListenerJobEnd","Job ID":20,"Job Result":{"Result":"JobSucceeded"}} + """ + + private val environmentUpdateJsonString = + """ + {"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"GC speed":"9999 objects/s", + "Java home":"Land of coffee"},"Spark Properties":{"Job throughput":"80000 jobs/s, + regardless of job type"},"System Properties":{"Username":"guest","Password":"guest"}, + "Classpath Entries":{"Super library":"/tmp/super_library"}} + """ + + private val blockManagerAddedJsonString = + """ + {"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"Stars", + "Host":"In your multitude...","Port":300,"Netty Port":400},"Maximum Memory":500} + """ + + private val blockManagerRemovedJsonString = + """ + {"Event":"SparkListenerBlockManagerRemoved","Block Manager ID":{"Executor ID":"Scarce", + "Host":"to be counted...","Port":100,"Netty Port":200}} + """ + + private val unpersistRDDJsonString = + """ + {"Event":"SparkListenerUnpersistRDD","RDD ID":12345} + """ + + } diff --git a/docs/configuration.md b/docs/configuration.md index a006224d5080c..16ee5ec0f230f 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -490,7 +490,30 @@ Apart from these, the following properties are also available, and may be useful + + + + + + + + + + + + + + +
    {s.stageId} - {poolName.get} + + {poolName.get} + + {description} {submissionTime} - {duration.map(d => parent.formatDuration(d)).getOrElse("Unknown")} - {formattedDuration} {makeProgressBar(startedTasks, completedTasks, failedTasks, totalTasks)}
    - + {rdd.name}
    {status.blockManagerId.host + ":" + status.blockManagerId.port} + {Utils.bytesToString(status.memUsedByRDD(rddId))} + ({Utils.bytesToString(status.memRemaining)} Remaining) + {Utils.bytesToString(status.diskUsedByRDD(rddId))}
    {id}
    {status.blockManagerId.host + ":" + status.blockManagerId.port} - {Utils.bytesToString(status.memUsedByRDD(rddId))} - ({Utils.bytesToString(status.memRemaining)} Remaining) - {Utils.bytesToString(status.diskUsedByRDD(rddId))}
    spark.logConf false - Log the supplied SparkConf as INFO at start of spark context. + Whether to log the supplied SparkConf as INFO at start of spark context. +
    spark.eventLog.enabledfalse + Whether to log spark events, useful for reconstructing the Web UI after the application has finished. +
    spark.eventLog.compressfalse + Whether to compress logged events, if spark.eventLog.enabled is true. +
    spark.eventLog.dirfile:///tmp/spark-events + Base directory in which spark events are logged, if spark.eventLog.enabled is true. + Within this base directory, Spark creates a sub-directory for each application, and logs the events + specific to the application in this directory.