aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--core/src/main/scala/org/apache/spark/Aggregator.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/CacheManager.scala76
-rw-r--r--core/src/main/scala/org/apache/spark/SecurityManager.scala8
-rw-r--r--core/src/main/scala/org/apache/spark/SparkContext.scala93
-rw-r--r--core/src/main/scala/org/apache/spark/SparkEnv.scala80
-rw-r--r--core/src/main/scala/org/apache/spark/TaskEndReason.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala5
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala1
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/Master.scala73
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala84
-rwxr-xr-xcore/src/main/scala/org/apache/spark/deploy/worker/Worker.scala18
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala84
-rw-r--r--core/src/main/scala/org/apache/spark/executor/Executor.scala8
-rw-r--r--core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala13
-rw-r--r--core/src/main/scala/org/apache/spark/io/CompressionCodec.scala5
-rw-r--r--core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala8
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/RDD.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala306
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala98
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala266
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/JobResult.scala5
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala101
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/Pool.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala104
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala176
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala167
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala29
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala9
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala5
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala5
-rw-r--r--core/src/main/scala/org/apache/spark/storage/BlockManager.scala241
-rw-r--r--core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala160
-rw-r--r--core/src/main/scala/org/apache/spark/storage/MemoryStore.scala100
-rw-r--r--core/src/main/scala/org/apache/spark/storage/PutResult.scala12
-rw-r--r--core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala94
-rw-r--r--core/src/main/scala/org/apache/spark/storage/StorageUtils.scala119
-rw-r--r--core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala7
-rw-r--r--core/src/main/scala/org/apache/spark/ui/JettyUtils.scala121
-rw-r--r--core/src/main/scala/org/apache/spark/ui/SparkUI.scala101
-rw-r--r--core/src/main/scala/org/apache/spark/ui/UIUtils.scala40
-rw-r--r--core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala108
-rw-r--r--core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala224
-rw-r--r--core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala48
-rw-r--r--core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala90
-rw-r--r--core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala277
-rw-r--r--core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala32
-rw-r--r--core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala29
-rw-r--r--core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala25
-rw-r--r--core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala248
-rw-r--r--core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala75
-rw-r--r--core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala74
-rw-r--r--core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala41
-rw-r--r--core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala86
-rw-r--r--core/src/main/scala/org/apache/spark/util/FileLogger.scala165
-rw-r--r--core/src/main/scala/org/apache/spark/util/JsonProtocol.scala710
-rw-r--r--core/src/main/scala/org/apache/spark/util/Utils.scala38
-rw-r--r--core/src/test/java/org/apache/spark/JavaAPISuite.java23
-rw-r--r--core/src/test/scala/org/apache/spark/CacheManagerSuite.scala11
-rw-r--r--core/src/test/scala/org/apache/spark/JobCancellationSuite.scala8
-rw-r--r--core/src/test/scala/org/apache/spark/PipedRDDSuite.scala5
-rw-r--r--core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala4
-rw-r--r--core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala11
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala15
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala126
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala150
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala8
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala8
-rw-r--r--core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala69
-rw-r--r--core/src/test/scala/org/apache/spark/ui/UISuite.scala22
-rw-r--r--core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala20
-rw-r--r--core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala559
-rw-r--r--docs/configuration.md25
84 files changed, 4268 insertions, 1944 deletions
diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala
index c4579cf6ad..ceead59b79 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 872e892c04..c7893f288b 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 591978c1d3..2237ee3bb7 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 852ed8fe1f..a1003b7925 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()
}
@@ -723,6 +745,23 @@ class SparkContext(
}
/**
+ * 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
* filesystems), an HTTP, HTTPS or FTP URI, or local:/path for a file on every worker node.
@@ -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 774cbd6441..a1af63fa4a 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 3fd6f5eb47..f1a753b6ab 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 8e0eab56a3..35508b6e5a 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 449b953530..15fa8a7679 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", "<unknown>")
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 cefb1ff97e..c4f5e294a3 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 e8867bc169..46b9f4dc7d 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 b8dfa44102..1fd2114169 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 90cad3c37f..cb092cb5d5 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) {
</li>
<li><strong>Submit Date:</strong> {app.submitDate}</li>
<li><strong>State:</strong> {app.state}</li>
- <li><strong><a href={app.appUiUrl}>Application Detail UI</a></strong></li>
+ <li><strong><a href={app.desc.appUiUrl}>Application Detail UI</a></strong></li>
</ul>
</div>
</div>
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 3233cd97f7..7ec71eb80b 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) {
<a href={"app?appId=" + app.id}>{app.id}</a>
</td>
<td>
- <a href={app.appUiUrl}>{app.desc.name}</a>
+ <a href={app.desc.appUiUrl}>{app.desc.name}</a>
</td>
<td>
{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 4ad1f95be3..bd75b2dfd0 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 afaabedffe..5e0fc31fff 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 4e33b330ad..de76a5d5eb 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 = <node>{Utils.offsetBytes(path, startByte, endByte)}</node>
-
val linkToMaster = <p><a href={worker.activeMasterWebUiUrl}>Back to Master</a></p>
-
val range = <span>Bytes {startByte.toString} - {endByte.toString} of {logLength}</span>
val backButton =
if (startByte > 0) {
<a href={"?%s&logType=%s&offset=%s&byteLength=%s"
- .format(params, logType, math.max(startByte-byteLength, 0), byteLength)}>
+ .format(params, logType, math.max(startByte - byteLength, 0), byteLength)}>
<button type="button" class="btn btn-default">
Previous {Utils.bytesToString(math.min(byteLength, startByte))}
</button>
@@ -146,7 +143,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I
<a href={"?%s&logType=%s&offset=%s&byteLength=%s".
format(params, logType, endByte, byteLength)}>
<button type="button" class="btn btn-default">
- Next {Utils.bytesToString(math.min(byteLength, logLength-endByte))}
+ Next {Utils.bytesToString(math.min(byteLength, logLength - endByte))}
</button>
</a>
}
@@ -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 2ea2ec29f5..8fe9b848ba 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 760458cb02..88625e79a5 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 848b5c439b..059e58824c 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 3110eccdee..854b52c510 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 ddb901246d..1b43040c6d 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 d83d0341c6..77c558ac46 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 39cd98e2d7..04c53d4684 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 b52fe2410a..5878e73390 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 0000000000..217f8825c2
--- /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 01cbcc390c..b3a67d7e17 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", "<unknown>"),
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 d94f6ad924..3cf4e3077e 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 b026f860a8..8007b54187 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 0000000000..353a48661b
--- /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 4bc13c23d9..187672c4e1 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 0000000000..db76178b65
--- /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 = "<not started>"
+ 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 9590c03f10..d4eb0ac88d 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 17b1328b86..729e120497 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 8f320e5c7a..8115a7ed78 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 ea3229b75b..308edb12ed 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 abff252597..30bceb47b9 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 ee4b65e312..25b7472a99 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 28b019d9fd..06b041e1fd 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 bcf0ce19a5..4092dd04b1 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 1bf3f4db32..71584b6eb1 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 98cd6e68fa..be537d7730 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 e531467ccc..ed6937851b 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 a999d76a32..ff2652b640 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 38836d44b0..488f1ea962 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 2eba2f06b5..f0eac7594e 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 0000000000..26565f56ad
--- /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 ("<driver>"). 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 "<driver>" to be consistent.
+ */
+ def formatExecutorId(execId: String): String = {
+ if (execId == "localhost") "<driver>" 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 2d88a40fbb..6153dfe0b7 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 36f2a0fd02..226ed2a132 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(
"<driver>", 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 e0555ca7ac..6e1736f6fb 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 5f0dee64fe..fd638c83aa 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 547a194d58..a487924eff 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 => <li class="active"><a href={prependBaseUri("/stages")}>Stages</a></li>
- case _ => <li><a href={prependBaseUri("/stages")}>Stages</a></li>
+ case Stages =>
+ <li class="active"><a href={prependBaseUri(basePath, "/stages")}>Stages</a></li>
+ case _ => <li><a href={prependBaseUri(basePath, "/stages")}>Stages</a></li>
}
val storage = page match {
- case Storage => <li class="active"><a href={prependBaseUri("/storage")}>Storage</a></li>
- case _ => <li><a href={prependBaseUri("/storage")}>Storage</a></li>
+ case Storage =>
+ <li class="active"><a href={prependBaseUri(basePath, "/storage")}>Storage</a></li>
+ case _ => <li><a href={prependBaseUri(basePath, "/storage")}>Storage</a></li>
}
val environment = page match {
case Environment =>
- <li class="active"><a href={prependBaseUri("/environment")}>Environment</a></li>
- case _ => <li><a href={prependBaseUri("/environment")}>Environment</a></li>
+ <li class="active"><a href={prependBaseUri(basePath, "/environment")}>Environment</a></li>
+ case _ => <li><a href={prependBaseUri(basePath, "/environment")}>Environment</a></li>
}
val executors = page match {
case Executors =>
- <li class="active"><a href={prependBaseUri("/executors")}>Executors</a></li>
- case _ => <li><a href={prependBaseUri("/executors")}>Executors</a></li>
+ <li class="active"><a href={prependBaseUri(basePath, "/executors")}>Executors</a></li>
+ case _ => <li><a href={prependBaseUri(basePath, "/executors")}>Executors</a></li>
}
<html>
@@ -58,14 +63,15 @@ private[spark] object UIUtils {
<meta http-equiv="Content-type" content="text/html; charset=utf-8" />
<link rel="stylesheet" href={prependBaseUri("/static/bootstrap.min.css")}
type="text/css" />
- <link rel="stylesheet" href={prependBaseUri("/static/webui.css")} type="text/css" />
+ <link rel="stylesheet" href={prependBaseUri("/static/webui.css")}
+ type="text/css" />
<script src={prependBaseUri("/static/sorttable.js")} ></script>
- <title>{sc.appName} - {title}</title>
+ <title>{appName} - {title}</title>
</head>
<body>
<div class="navbar navbar-static-top">
<div class="navbar-inner">
- <a href={prependBaseUri("/")} class="brand">
+ <a href={prependBaseUri(basePath, "/")} class="brand">
<img src={prependBaseUri("/static/spark-logo-77x50px-hd.png")} />
</a>
<ul class="nav">
@@ -74,7 +80,7 @@ private[spark] object UIUtils {
{environment}
{executors}
</ul>
- <p class="navbar-text pull-right"><strong>{sc.appName}</strong> application UI</p>
+ <p class="navbar-text pull-right"><strong>{appName}</strong> application UI</p>
</div>
</div>
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 14333476c0..23e90c34d5 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)) = <tr><td>{kv._1}</td><td>{kv._2}</td></tr>
- 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)) = <tr><td>{kv._1}</td><td>{kv._2}</td></tr>
- 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)) = <tr><td>{data._1}</td><td>{data._2}</td></tr>
- 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 =
<span>
- <h4>Runtime Information</h4> {jvmTable}
- <h4>Spark Properties</h4>
- {sparkPropertyTable}
- <h4>System Properties</h4>
- {otherPropertyTable}
- <h4>Classpath Entries</h4>
- {classPathTable}
+ <h4>Runtime Information</h4> {runtimeInformationTable}
+ <h4>Spark Properties</h4> {sparkPropertiesTable}
+ <h4>System Properties</h4> {systemPropertiesTable}
+ <h4>Classpath Entries</h4> {classpathEntriesTable}
</span>
- 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)) = <tr><td>{kv._1}</td><td>{kv._2}</td></tr>
+ private def propertyRow(kv: (String, String)) = <tr><td>{kv._1}</td><td>{kv._2}</td></tr>
+ private def classPathRow(data: (String, String)) = <tr><td>{data._1}</td><td>{data._2}</td></tr>
+}
+
+/**
+ * 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 4235cfeff9..031ed88a49 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]) = {
- <tr>
- <td>{kv(0)}</td>
- <td>{kv(1)}</td>
- <td>{kv(2)}</td>
- <td sorttable_customkey={kv(3)}>
- {Utils.bytesToString(kv(3).toLong)} / {Utils.bytesToString(kv(4).toLong)}
- </td>
- <td sorttable_customkey={kv(5)}>
- {Utils.bytesToString(kv(5).toLong)}
- </td>
- <td>{kv(6)}</td>
- <td>{kv(7)}</td>
- <td>{kv(8)}</td>
- <td>{kv(9)}</td>
- <td>{Utils.msDurationToString(kv(10).toLong)}</td>
- <td>{Utils.bytesToString(kv(11).toLong)}</td>
- <td>{Utils.bytesToString(kv(12).toLong)}</td>
- </tr>
- }
-
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 =
<div class="row-fluid">
@@ -100,18 +74,61 @@ private[spark] class ExecutorsUI(val sc: SparkContext) {
</div>
</div>;
- 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")
+ <tr>
+ <td>{values("Executor ID")}</td>
+ <td>{values("Address")}</td>
+ <td>{values("RDD Blocks")}</td>
+ <td sorttable_customkey={memoryUsed}>
+ {Utils.bytesToString(memoryUsed.toLong)} /
+ {Utils.bytesToString(maximumMemory.toLong)}
+ </td>
+ <td sorttable_customkey={diskUsed}>
+ {Utils.bytesToString(diskUsed.toLong)}
+ </td>
+ <td>{values("Active Tasks")}</td>
+ <td>{values("Failed Tasks")}</td>
+ <td>{values("Complete Tasks")}</td>
+ <td>{values("Total Tasks")}</td>
+ <td>{Utils.msDurationToString(values("Task Time").toLong)}</td>
+ <td>{Utils.bytesToString(values("Shuffle Read").toLong)}</td>
+ <td>{Utils.bytesToString(values("Shuffle Write").toLong)}</td>
+ </tr>
}
- 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 64e22a30b4..1dfe1d4f1f 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 d012ba4dbb..73861ae674 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) => {
- <tr>
- <td>{k}</td>
- <td>{executorIdToAddress.getOrElse(k, "CANNOT FIND ADDRESS")}</td>
- <td>{parent.formatDuration(v.taskTime)}</td>
- <td>{v.failedTasks + v.succeededTasks}</td>
- <td>{v.failedTasks}</td>
- <td>{v.succeededTasks}</td>
- <td>{Utils.bytesToString(v.shuffleRead)}</td>
- <td>{Utils.bytesToString(v.shuffleWrite)}</td>
- <td>{Utils.bytesToString(v.memoryBytesSpilled)}</td>
- <td>{Utils.bytesToString(v.diskBytesSpilled)}</td>
- </tr>
- }
+ case Some(x) =>
+ x.toSeq.sortBy(_._1).map { case (k, v) => {
+ <tr>
+ <td>{k}</td>
+ <td>{executorIdToAddress.getOrElse(k, "CANNOT FIND ADDRESS")}</td>
+ <td>{parent.formatDuration(v.taskTime)}</td>
+ <td>{v.failedTasks + v.succeededTasks}</td>
+ <td>{v.failedTasks}</td>
+ <td>{v.succeededTasks}</td>
+ <td>{Utils.bytesToString(v.shuffleRead)}</td>
+ <td>{Utils.bytesToString(v.shuffleWrite)}</td>
+ <td>{Utils.bytesToString(v.memoryBytesSpilled)}</td>
+ <td>{Utils.bytesToString(v.diskBytesSpilled)}</td>
+ </tr>
}
}
- 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 81713edcf5..f3c93d4214 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 =
- <div>
- <ul class="unstyled">
- <li>
- <strong>Total Duration: </strong>
- {parent.formatDuration(now - listener.sc.startTime)}
- </li>
- <li><strong>Scheduling Mode:</strong> {parent.sc.getSchedulingMode}</li>
- <li>
- <a href="#active"><strong>Active Stages:</strong></a>
- {activeStages.size}
- </li>
- <li>
- <a href="#completed"><strong>Completed Stages:</strong></a>
- {completedStages.size}
- </li>
- <li>
+ <div>
+ <ul class="unstyled">
+ {if (live) {
+ // Total duration is not meaningful unless the UI is live
+ <li>
+ <strong>Total Duration: </strong>
+ {parent.formatDuration(now - sc.startTime)}
+ </li>
+ }}
+ <li>
+ <strong>Scheduling Mode: </strong>
+ {listener.schedulingMode.map(_.toString).getOrElse("Unknown")}
+ </li>
+ <li>
+ <a href="#active"><strong>Active Stages:</strong></a>
+ {activeStages.size}
+ </li>
+ <li>
+ <a href="#completed"><strong>Completed Stages:</strong></a>
+ {completedStages.size}
+ </li>
+ <li>
<a href="#failed"><strong>Failed Stages:</strong></a>
- {failedStages.size}
- </li>
- </ul>
- </div>
+ {failedStages.size}
+ </li>
+ </ul>
+ </div>
val content = summary ++
- {if (listener.sc.getSchedulingMode == SchedulingMode.FAIR) {
- <h4>{pools.size} Fair Scheduler Pools</h4> ++ poolTable.toNodeSeq
+ {if (live && isFairScheduler) {
+ <h4>{pools.size} Fair Scheduler Pools</h4> ++ poolTable.toNodeSeq
} else {
- Seq()
+ Seq[Node]()
}} ++
<h4 id="active">Active Stages ({activeStages.size})</h4> ++
- activeStagesTable.toNodeSeq++
+ activeStagesTable.toNodeSeq ++
<h4 id="completed">Completed Stages ({completedStages.size})</h4> ++
- completedStagesTable.toNodeSeq++
+ completedStagesTable.toNodeSeq ++
<h4 id ="failed">Failed Stages ({failedStages.size})</h4> ++
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 07a08f5277..d10aa12b9e 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 2d95d47e15..ee4e9c69c1 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 eb7518a020..bd33182b70 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 = <h4>Summary </h4> ++ poolTable.toNodeSeq() ++
- <h4>{activeStages.size} Active Stages</h4> ++ activeStagesTable.toNodeSeq()
+ val content =
+ <h4>Summary </h4> ++ poolTable.toNodeSeq ++
+ <h4>{activeStages.size} Active Stages</h4> ++ 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 22bc97ada1..c5c8d86687 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] = {
<table class="table table-bordered table-striped table-condensed sortable table-fixed">
<thead>
<th>Pool Name</th>
@@ -53,15 +53,18 @@ private[spark] class PoolTable(pools: Seq[Schedulable], listener: JobProgressLis
</table>
}
- 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
}
<tr>
<td>
- <a href={"%s/stages/pool?poolname=%s".format(UIUtils.prependBaseUri(),p.name)}>{p.name}</a>
+ <a href={"%s/stages/pool?poolname=%s".format(UIUtils.prependBaseUri(basePath), p.name)}>
+ {p.name}
+ </a>
</td>
<td>{p.minShare}</td>
<td>{p.weight}</td>
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 ddc687a45a..da7f202330 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 =
<div>
<h4>Summary Metrics</h4> No tasks have started yet
<h4>Tasks</h4> No tasks have started yet
</div>
- 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 =
<div>
@@ -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] = <tr> {data.map(d => <td>{d}</td>)} </tr>
- 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 ++
<h4>Summary Metrics for {numCompleted} Completed Tasks</h4> ++
<div>{summaryTable.getOrElse("No tasks have reported metrics yet.")}</div> ++
- <h4>Aggregated Metrics by Executor</h4> ++ executorTable.toNodeSeq() ++
+ <h4>Aggregated Metrics by Executor</h4> ++ executorTable.toNodeSeq ++
<h4>Tasks</h4> ++ 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 => <span style="display:block;">{e.toString}</span>)
- 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("")
-
- <tr>
- <td>{info.index}</td>
- <td>{info.taskId}</td>
- <td>{info.status}</td>
- <td>{info.taskLocality}</td>
- <td>{info.host}</td>
- <td>{dateFmt.format(new Date(info.launchTime))}</td>
- <td sorttable_customkey={duration.toString}>
- {formatDuration}
- </td>
- <td sorttable_customkey={gcTime.toString}>
- {if (gcTime > 0) parent.formatDuration(gcTime) else ""}
- </td>
- <td sorttable_customkey={serializationTime.toString}>
- {if (serializationTime > 0) parent.formatDuration(serializationTime) else ""}
- </td>
- {if (shuffleRead) {
- <td sorttable_customkey={shuffleReadSortable}>
- {shuffleReadReadable}
- </td>
- }}
- {if (shuffleWrite) {
- <td sorttable_customkey={writeTimeSortable}>
- {writeTimeReadable}
- </td>
- <td sorttable_customkey={shuffleWriteSortable}>
- {shuffleWriteReadable}
- </td>
- }}
- {if (bytesSpilled) {
- <td sorttable_customkey={memoryBytesSpilledSortable}>
- {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("")
+
+ <tr>
+ <td>{info.index}</td>
+ <td>{info.taskId}</td>
+ <td>{info.status}</td>
+ <td>{info.taskLocality}</td>
+ <td>{info.host}</td>
+ <td>{dateFmt.format(new Date(info.launchTime))}</td>
+ <td sorttable_customkey={duration.toString}>
+ {formatDuration}
+ </td>
+ <td sorttable_customkey={gcTime.toString}>
+ {if (gcTime > 0) parent.formatDuration(gcTime) else ""}
</td>
- <td sorttable_customkey={diskBytesSpilledSortable}>
- {diskBytesSpilledReadable}
+ <td sorttable_customkey={serializationTime.toString}>
+ {if (serializationTime > 0) parent.formatDuration(serializationTime) else ""}
</td>
- }}
- <td>{exception.map(e =>
- <span>
- {e.className} ({e.description})<br/>
- {fmtStackTrace(e.stackTrace)}
- </span>).getOrElse("")}
- </td>
- </tr>
+ {if (shuffleRead) {
+ <td sorttable_customkey={shuffleReadSortable}>
+ {shuffleReadReadable}
+ </td>
+ }}
+ {if (shuffleWrite) {
+ <td sorttable_customkey={writeTimeSortable}>
+ {writeTimeReadable}
+ </td>
+ <td sorttable_customkey={shuffleWriteSortable}>
+ {shuffleWriteReadable}
+ </td>
+ }}
+ {if (bytesSpilled) {
+ <td sorttable_customkey={memoryBytesSpilledSortable}>
+ {memoryBytesSpilledReadable}
+ </td>
+ <td sorttable_customkey={diskBytesSpilledSortable}>
+ {diskBytesSpilledReadable}
+ </td>
+ }}
+ <td>
+ {exception.map { e =>
+ <span>
+ {e.className} ({e.description})<br/>
+ {fmtStackTrace(e.stackTrace)}
+ </span>
+ }.getOrElse("")}
+ </td>
+ </tr>
+ }
}
}
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 99828487f1..68fef5234c 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] = {
<table class="table table-bordered table-striped table-condensed sortable">
<thead>
@@ -72,55 +72,56 @@ private[spark] class StageTable(val stages: Seq[StageInfo], val parent: JobProgr
</div>
}
-
+ /** Render an HTML row that represents a stage */
private def stageRow(s: StageInfo): Seq[Node] = {
+ val poolName = listener.stageIdToPool.get(s.stageId)
+ val nameLink =
+ <a href={"%s/stages/stage?id=%s".format(UIUtils.prependBaseUri(basePath), s.stageId)}>
+ {s.name}
+ </a>
+ val description = listener.stageIdToDescription.get(s.stageId)
+ .map(d => <div><em>{d}</em></div><div>{nameLink}</div>).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 =
- <a href={"%s/stages/stage?id=%s".format(UIUtils.prependBaseUri(),s.stageId)}>{s.name}</a>
- val description = listener.stageIdToDescription.get(s.stageId)
- .map(d => <div><em>{d}</em></div><div>{nameLink}</div>).getOrElse(nameLink)
- val finishTime = s.completionTime.getOrElse(System.currentTimeMillis())
- val duration = s.submissionTime.map { t =>
- if (finishTime > t) finishTime - t else System.currentTimeMillis - t
- }
-
<tr>
<td>{s.stageId}</td>
{if (isFairScheduler) {
- <td><a href={"%s/stages/pool?poolname=%s".format(UIUtils.prependBaseUri(),poolName.get)}>
- {poolName.get}</a></td>}
- }
+ <td>
+ <a href={"%s/stages/pool?poolname=%s"
+ .format(UIUtils.prependBaseUri(basePath), poolName.get)}>
+ {poolName.get}
+ </a>
+ </td>
+ }}
<td>{description}</td>
<td valign="middle">{submissionTime}</td>
- <td sorttable_customkey={duration.getOrElse(-1).toString}>
- {duration.map(d => parent.formatDuration(d)).getOrElse("Unknown")}
- </td>
+ <td sorttable_customkey={duration.getOrElse(-1).toString}>{formattedDuration}</td>
<td class="progress-cell">
{makeProgressBar(startedTasks, completedTasks, failedTasks, totalTasks)}
</td>
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 cb2083eb01..4d8b01dbe6 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 6a3c41fb11..b2732de510 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] = {
<tr>
<td>
- <a href={"%s/storage/rdd?id=%s".format(prependBaseUri(),rdd.id)}>
+ <a href={"%s/storage/rdd?id=%s".format(UIUtils.prependBaseUri(basePath), rdd.id)}>
{rdd.name}
</a>
</td>
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 78b149b14b..3f42eba4ec 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 =
<div class="row-fluid">
@@ -94,10 +95,39 @@ private[spark] class RDDPage(parent: BlockManagerUI) {
</div>
</div>;
- 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
+ <tr>
+ <td>{status.blockManagerId.host + ":" + status.blockManagerId.port}</td>
+ <td>
+ {Utils.bytesToString(status.memUsedByRDD(rddId))}
+ ({Utils.bytesToString(status.memRemaining)} Remaining)
+ </td>
+ <td>{Utils.bytesToString(status.diskUsedByRDD(rddId))}</td>
+ </tr>
}
- 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
<tr>
<td>{id}</td>
@@ -115,16 +145,4 @@ private[spark] class RDDPage(parent: BlockManagerUI) {
</td>
</tr>
}
-
- def workerRow(worker: (Int, StorageStatus)): Seq[Node] = {
- val (rddId, status) = worker
- <tr>
- <td>{status.blockManagerId.host + ":" + status.blockManagerId.port}</td>
- <td>
- {Utils.bytesToString(status.memUsedByRDD(rddId))}
- ({Utils.bytesToString(status.memRemaining)} Remaining)
- </td>
- <td>{Utils.bytesToString(status.diskUsedByRDD(rddId))}</td>
- </tr>
- }
}
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 0000000000..f07962096a
--- /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 0000000000..346f2b7856
--- /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 38a275d438..13d9dbdd9a 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 40e853c39c..c6b65c7348 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.JavaPairRDD;
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 class JavaAPISuite implements Serializable {
@Test
public void javaDoubleRDDHistoGram() {
- JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0));
- // Test using generated buckets
- Tuple2<double[], long[]> 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<double[], long[]> 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 class JavaAPISuite implements Serializable {
@Test
public void iterator() {
JavaRDD<Integer> 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 ea936e815b..b86923f07f 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 20c503d30c..7a39d1af9e 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 0bac78d8a6..6e7fd55fa4 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 3bb936790d..b543471a5d 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 bae3b37e26..9f2924c23b 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", "<unknown>"),
- 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 ad890b4e4d..c97543f57d 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 25fe63c265..0000000000
--- 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 7c4f2b4361..a25ce35736 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 6b0800af9c..9274e01632 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 73153d23c4..9af5d3a303 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 1036b9f34e..e83cd55e73 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("<driver>", 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("<driver>", 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 30415814ad..45c3224279 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 8ca863e8b3..d8a3e859f8 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 0000000000..67c0a434c9
--- /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 a006224d50..16ee5ec0f2 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
<td>spark.logConf</td>
<td>false</td>
<td>
- Log the supplied SparkConf as INFO at start of spark context.
+ Whether to log the supplied SparkConf as INFO at start of spark context.
+ </td>
+</tr>
+<tr>
+ <td>spark.eventLog.enabled</td>
+ <td>false</td>
+ <td>
+ Whether to log spark events, useful for reconstructing the Web UI after the application has finished.
+ </td>
+</tr>
+<tr>
+ <td>spark.eventLog.compress</td>
+ <td>false</td>
+ <td>
+ Whether to compress logged events, if <code>spark.eventLog.enabled</code> is true.
+ </td>
+</tr>
+<tr>
+ <td>spark.eventLog.dir</td>
+ <td>file:///tmp/spark-events</td>
+ <td>
+ Base directory in which spark events are logged, if <code>spark.eventLog.enabled</code> 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.
</td>
</tr>
<tr>