diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala
index c4579cf6ad560cbbf2037bd1c217c7946b14b06f..ceead59b79ed6aa97431af42d18332436a2660fb 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 872e892c04fe6e20dfd6ec19d4e84f90f5dc9a31..c7893f288b4b5f606b9cb917a001dce3dfe13b81 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 591978c1d363060a341eb7be72660e401653f801..2237ee3bb7aadf32d0c6c5a7c38ddd4528a71366 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 852ed8fe1fb9147b83f8389401c92f2aa39b18af..a1003b79257153dd843a3217ef1d35170e15b813 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -132,6 +132,9 @@ class SparkContext(
 
   if (master == "yarn-client") System.setProperty("SPARK_YARN_MODE", "true")
 
+  // An asynchronous listener bus for Spark events
+  private[spark] val listenerBus = new LiveListenerBus
+
   // Create the Spark execution environment (cache, map output tracker, etc)
   private[spark] val env = SparkEnv.create(
     conf,
@@ -139,7 +142,8 @@ class SparkContext(
     conf.get("spark.driver.host"),
     conf.get("spark.driver.port").toInt,
     isDriver = true,
-    isLocal = isLocal)
+    isLocal = isLocal,
+    listenerBus = listenerBus)
   SparkEnv.set(env)
 
   // Used to store a URL for each static file/jar together with the file's local timestamp
@@ -151,9 +155,26 @@ class SparkContext(
   private[spark] val metadataCleaner =
     new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup, conf)
 
-  // Initialize the Spark UI
+  // Initialize the Spark UI, registering all associated listeners
   private[spark] val ui = new SparkUI(this)
   ui.bind()
+  ui.start()
+
+  // Optionally log Spark events
+  private[spark] val eventLogger: Option[EventLoggingListener] = {
+    if (conf.getBoolean("spark.eventLog.enabled", false)) {
+      val logger = new EventLoggingListener(appName, conf)
+      listenerBus.addListener(logger)
+      Some(logger)
+    } else None
+  }
+
+  // Information needed to replay logged events, if any
+  private[spark] val eventLoggingInfo: Option[EventLoggingInfo] =
+    eventLogger.map { logger => Some(logger.info) }.getOrElse(None)
+
+  // At this point, all relevant SparkListeners have been registered, so begin releasing events
+  listenerBus.start()
 
   val startTime = System.currentTimeMillis()
 
@@ -200,13 +221,13 @@ class SparkContext(
   executorEnvs("SPARK_USER") = sparkUser
 
   // Create and start the scheduler
-  private[spark] var taskScheduler = SparkContext.createTaskScheduler(this, master, appName)
+  private[spark] var taskScheduler = SparkContext.createTaskScheduler(this, master)
   taskScheduler.start()
 
-  @volatile private[spark] var dagScheduler = new DAGScheduler(taskScheduler)
+  @volatile private[spark] var dagScheduler = new DAGScheduler(this)
   dagScheduler.start()
 
-  ui.start()
+  postEnvironmentUpdate()
 
   /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */
   val hadoopConfiguration = {
@@ -571,7 +592,6 @@ class SparkContext(
       .flatMap(x => Utils.deserialize[Array[T]](x._2.getBytes))
   }
 
-
   protected[spark] def checkpointFile[T: ClassTag](
       path: String
     ): RDD[T] = {
@@ -641,10 +661,11 @@ class SparkContext(
     Utils.fetchFile(path, new File(SparkFiles.getRootDirectory), conf, env.securityManager)
 
     logInfo("Added file " + path + " at " + key + " with timestamp " + addedFiles(key))
+    postEnvironmentUpdate()
   }
 
   def addSparkListener(listener: SparkListener) {
-    dagScheduler.addSparkListener(listener)
+    listenerBus.addListener(listener)
   }
 
   /**
@@ -671,7 +692,7 @@ class SparkContext(
    */
   def getPersistentRDDs: Map[Int, RDD[_]] = persistentRdds.toMap
 
-  def getStageInfo: Map[Stage,StageInfo] = {
+  def getStageInfo: Map[Stage, StageInfo] = {
     dagScheduler.stageToInfos
   }
 
@@ -698,7 +719,7 @@ class SparkContext(
   }
 
   /**
-   *  Return current scheduling mode
+   * Return current scheduling mode
    */
   def getSchedulingMode: SchedulingMode.SchedulingMode = {
     taskScheduler.schedulingMode
@@ -708,6 +729,7 @@ class SparkContext(
    * Clear the job's list of files added by `addFile` so that they do not get downloaded to
    * any new nodes.
    */
+  @deprecated("adding files no longer creates local copies that need to be deleted", "1.0.0")
   def clearFiles() {
     addedFiles.clear()
   }
@@ -722,6 +744,23 @@ class SparkContext(
     dagScheduler.getPreferredLocs(rdd, partition)
   }
 
+  /**
+   * Register an RDD to be persisted in memory and/or disk storage
+   */
+  private[spark] def persistRDD(rdd: RDD[_]) {
+    persistentRdds(rdd.id) = rdd
+  }
+
+  /**
+   * Unpersist an RDD from memory and/or disk storage
+   */
+  private[spark] def unpersistRDD(rdd: RDD[_], blocking: Boolean = true) {
+    val rddId = rdd.id
+    env.blockManager.master.removeRdd(rddId, blocking)
+    persistentRdds.remove(rddId)
+    listenerBus.post(SparkListenerUnpersistRDD(rddId))
+  }
+
   /**
    * Adds a JAR dependency for all tasks to be executed on this SparkContext in the future.
    * The `path` passed can be either a local file, a file in HDFS (or other Hadoop-supported
@@ -744,7 +783,7 @@ class SparkContext(
             if (SparkHadoopUtil.get.isYarnMode() &&
                 (master == "yarn-standalone" || master == "yarn-cluster")) {
               // In order for this to work in yarn-cluster mode the user must specify the
-              // --addjars option to the client to upload the file into the distributed cache 
+              // --addjars option to the client to upload the file into the distributed cache
               // of the AM to make it show up in the current working directory.
               val fileName = new Path(uri.getPath).getName()
               try {
@@ -752,7 +791,7 @@ class SparkContext(
               } catch {
                 case e: Exception => {
                   // For now just log an error but allow to go through so spark examples work.
-                  // The spark examples don't really need the jar distributed since its also 
+                  // The spark examples don't really need the jar distributed since its also
                   // the app jar.
                   logError("Error adding jar (" + e + "), was the --addJars option used?")
                   null
@@ -773,12 +812,14 @@ class SparkContext(
         logInfo("Added JAR " + path + " at " + key + " with timestamp " + addedJars(key))
       }
     }
+    postEnvironmentUpdate()
   }
 
   /**
    * Clear the job's list of JARs added by `addJar` so that they do not get downloaded to
    * any new nodes.
    */
+  @deprecated("adding jars no longer creates local copies that need to be deleted", "1.0.0")
   def clearJars() {
     addedJars.clear()
   }
@@ -786,6 +827,7 @@ class SparkContext(
   /** Shut down the SparkContext. */
   def stop() {
     ui.stop()
+    eventLogger.foreach(_.stop())
     // Do this only if not stopped already - best case effort.
     // prevent NPE if stopped more than once.
     val dagSchedulerCopy = dagScheduler
@@ -793,12 +835,10 @@ class SparkContext(
     if (dagSchedulerCopy != null) {
       metadataCleaner.cancel()
       dagSchedulerCopy.stop()
+      listenerBus.stop()
       taskScheduler = null
       // TODO: Cache.stop()?
       env.stop()
-      // Clean up locally linked files
-      clearFiles()
-      clearJars()
       SparkEnv.set(null)
       ShuffleMapTask.clearCache()
       ResultTask.clearCache()
@@ -1026,6 +1066,19 @@ class SparkContext(
   /** Register a new RDD, returning its RDD ID */
   private[spark] def newRddId(): Int = nextRddId.getAndIncrement()
 
+  /** Post the environment update event once the task scheduler is ready */
+  private def postEnvironmentUpdate() {
+    if (taskScheduler != null) {
+      val schedulingMode = getSchedulingMode.toString
+      val addedJarPaths = addedJars.keys.toSeq
+      val addedFilePaths = addedFiles.keys.toSeq
+      val environmentDetails =
+        SparkEnv.environmentDetails(conf, schedulingMode, addedJarPaths, addedFilePaths)
+      val environmentUpdate = SparkListenerEnvironmentUpdate(environmentDetails)
+      listenerBus.post(environmentUpdate)
+    }
+  }
+
   /** Called by MetadataCleaner to clean up the persistentRdds map periodically */
   private[spark] def cleanup(cleanupTime: Long) {
     persistentRdds.clearOldValues(cleanupTime)
@@ -1189,9 +1242,7 @@ object SparkContext extends Logging {
   }
 
   /** Creates a task scheduler based on a given master URL. Extracted for testing. */
-  private def createTaskScheduler(sc: SparkContext, master: String, appName: String)
-      : TaskScheduler =
-  {
+  private def createTaskScheduler(sc: SparkContext, master: String): TaskScheduler = {
     // Regular expression used for local[N] master format
     val LOCAL_N_REGEX = """local\[([0-9]+)\]""".r
     // Regular expression for local[N, maxRetries], used in tests with failing tasks
@@ -1230,7 +1281,7 @@ object SparkContext extends Logging {
       case SPARK_REGEX(sparkUrl) =>
         val scheduler = new TaskSchedulerImpl(sc)
         val masterUrls = sparkUrl.split(",").map("spark://" + _)
-        val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls, appName)
+        val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls)
         scheduler.initialize(backend)
         scheduler
 
@@ -1247,7 +1298,7 @@ object SparkContext extends Logging {
         val localCluster = new LocalSparkCluster(
           numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt)
         val masterUrls = localCluster.start()
-        val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls, appName)
+        val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls)
         scheduler.initialize(backend)
         backend.shutdownCallback = (backend: SparkDeploySchedulerBackend) => {
           localCluster.stop()
@@ -1307,9 +1358,9 @@ object SparkContext extends Logging {
         val coarseGrained = sc.conf.getBoolean("spark.mesos.coarse", false)
         val url = mesosUrl.stripPrefix("mesos://") // strip scheme from raw Mesos URLs
         val backend = if (coarseGrained) {
-          new CoarseMesosSchedulerBackend(scheduler, sc, url, appName)
+          new CoarseMesosSchedulerBackend(scheduler, sc, url)
         } else {
-          new MesosSchedulerBackend(scheduler, sc, url, appName)
+          new MesosSchedulerBackend(scheduler, sc, url)
         }
         scheduler.initialize(backend)
         scheduler
diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala
index 774cbd6441a48eb24ba9f162df9929b5e11124eb..a1af63fa4a391b4e2a73f310a2009b7a6c5ce404 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 3fd6f5eb472f48ec9ddf4958ef50340e001102f5..f1a753b6ab8a97082a07205aedd825518bd1d947 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 8e0eab56a3dcfa319db40c03d2451e7782ef7344..35508b6e5acbabb76b7f07ebc9e55600390ce394 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 449b953530ff99a7862dc6fbd2a825153d0a75be..15fa8a76798741c430bc37cf1751506e928d5501 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 cefb1ff97e83c74603bfffe3c5f637cea1fe46f9..c4f5e294a393ea4ddbffc8d2d10597964ec9db2c 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 e8867bc1691d35aa279e0d68d4fe234ce9b42a7d..46b9f4dc7d3bae01723f699cb81e6daabbe87b8f 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 b8dfa441025834923a832e9886b9a762ad7796e9..1fd211416976e9fbb89f9e44e3a46969710a8f6b 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 90cad3c37fda6fd613fcc05648444ad4dcffa7e8..cb092cb5d576b9370606bd19e1b53f516ddf0ef9 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 3233cd97f7bd078a9bd9d6d29a52dda4ab3bf7ce..7ec71eb80bfc07fd4f0a17e11813c7410dda2687 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 4ad1f95be31c9901384edbdc512550decb81f103..bd75b2dfd0e0772663f2e5c11ae680421f1f4fa1 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 afaabedffefea4e186602e63f953674410c597be..5e0fc31fff22f57f5273c0e7868e1068f54aefc0 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 4e33b330ad4e7f9a653cc0d549bfb9069c7df8a1..de76a5d5eb7bc5af49a1445779a736dbfcb24473 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 2ea2ec29f59f57c1d9e30df2ddafb8c94ffbbb3b..8fe9b848ba1455aa437a1b906b07fca1a3a53aeb 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 760458cb02a9b3b8e6c7323db6c725e2f7bc477f..88625e79a5c68ac345da7d5cf89d6c1ed3d943ff 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 848b5c439bb5b2cebf306243dab4b7bd9dd2fc17..059e58824c39b62d627ad099a3c42a2048ea5733 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 3110eccdee4fcdb105594142c1106abdc846dc38..854b52c510e3d9206af180cfb3c8e7a71b8eaedb 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 ddb901246d3609935442ebb69bc75e6c1f7749b2..1b43040c6d9185024b520a9ea86be6f2d35512fa 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 d83d0341c61ab1e646b38c5196bafedaaf309948..77c558ac46f6f647124d312a50e2ecc2ebd93534 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 39cd98e2d74e4e1dc862a80201a3f47835259a24..04c53d468465a3edb1f84d7ead7f3c52443baaf8 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 b52fe2410abdeab499d69f12f1f752020d5d6eaa..5878e733908f5752179839f2afc20643c984fc98 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 0000000000000000000000000000000000000000..217f8825c2ae93df574f62273b532c4a5a37428e
--- /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 01cbcc390c6cda98a27abf7a740e6fd99f63ae09..b3a67d7e179763d4c0b731844659b2423c9f6d19 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 d94f6ad924260c40a07cd27a77fb3e2681f2046b..3cf4e3077e4a4cbe13fcec0075c258599a9ccec6 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 b026f860a8cd82ca6f61da52a68cd753c84ffd9d..8007b5418741e20caf963b542221c2d1844dfa8e 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 0000000000000000000000000000000000000000..353a48661b0f7992f215f3ec7a909037839745af
--- /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 4bc13c23d980be000d5fc3c3cad9216aab51d680..187672c4e19e7935bb3053e1559c01b4cd42e13b 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 0000000000000000000000000000000000000000..db76178b65501f2b614732fdee1e15a03203518e
--- /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 9590c03f10632f44382497c0393ea92fccb11720..d4eb0ac88d8e8f00391c241020655eb5dfc08799 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 17b1328b86788b1cae1ed3654fd5f8954fdf494a..729e1204975719ac1fc1ad74db8ee19e9cd1e253 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 8f320e5c7a74bd2db1f44c3b3b224ee3ca658f41..8115a7ed7896d575c567f9c04040b27fa00263c6 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 ea3229b75be36adb96d11afc5ef9ce97e32e929c..308edb12edd5c1986c4a23795a7f74150e37d304 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 abff252597e1612791b44f00e4334b94d4bbf4b9..30bceb47b9e7dd48dea86f746c1d7d414711f5b5 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 ee4b65e312abc8cbade3d1ff80f55f978c41a423..25b7472a99cdb6ed60029477acff318fe14b5eec 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 28b019d9fd4954afcc7f3891720e65556354531a..06b041e1fd9a91d94170e2d32a5f01f92ce80ac3 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 bcf0ce19a54cd68634c643aca64e9c22173f606e..4092dd04b112b9ed2456546c412a3b49e1d80293 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 1bf3f4db32ea74d6b01b4144a7189bec5883ed25..71584b6eb102a4813a11e1f202025688bad62193 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 98cd6e68fa7241dec3017d85153d57126196ca4e..be537d77309bc0235fb04769559ab32cfd2355b1 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 e531467cccb40567fd3adfaa7388874c21ff63d1..ed6937851b836da6b08c2c653827527c83d221ee 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 a999d76a326a68e12b59f1ffe49aa4dfa5bc7f05..ff2652b640272e5fe2c0ecccff91c0bff7bf4fc6 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 38836d44b04e863d95140eafb8a9f7fe227eda0f..488f1ea9628f5289b89f9a09b57596417b8ebaf1 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 2eba2f06b5bfd96e02260525d6e3309dd84d2919..f0eac7594ecf6ec194e94973a686462edc2853c1 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 0000000000000000000000000000000000000000..26565f56ad8580358126fb57ced2e6c4d219bd18
--- /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 2d88a40fbb3f24b4c09b20f6c79cbb485990bbf1..6153dfe0b7e1350438da529f2e2fb3a4835ca287 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 36f2a0fd02724196f27d2c606e34596c9a6ad68a..226ed2a132b00b8e4f809a39d814a9f486db9364 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 e0555ca7ac02f6ec55f1e5f2f3ce989cb16ce209..6e1736f6fbc23b514022947c82a74e62df267e0c 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 5f0dee64fedb7f060db45104595616fc11fe74e3..fd638c83aac6e331bb970da93b7551824800e8ee 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 547a194d58a5cb8f40f7176fb1efed398d3c5612..a487924effbff7dda54b05a3ae031a434515471f 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 14333476c0e3109af5c4b4bf041bbca1524c3533..23e90c34d5b33429335cf8353a4c79ab99842e7b 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 4235cfeff9fa20c3635d2ad58c900cd31b2ebfaf..031ed88a493a8305515e828eccb7f727d43c90e3 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 64e22a30b48f9380c47932d82e04d877d5f554f4..1dfe1d4f1fa11fff42c9b0b59ed8d154a96c01f9 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 d012ba4dbb3dbd7db6ed7a131ec9d291d3e99248..73861ae6746dab114c9f7328fa75489a2107d60c 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 81713edcf5db2e188be596f53ea6c3c4693a45ea..f3c93d4214ad02352ae6868f816d1a88b45c8dd0 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 07a08f5277d1972596930fff2d9dfb140b05acc0..d10aa12b9ebca3f1e002d24bc5ae2683ebdc529b 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 2d95d47e154cd254ebb01eb9af6bb297904b27e0..ee4e9c69c1bd1df5efd1458cd4b8e0aed4a1c8d9 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 eb7518a0208405eaa6e0cb71e2799f2f4463b0be..bd33182b700596b89255e522bf9eb08b1701af9a 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 22bc97ada18be789e5bf25c63b13377941cf2176..c5c8d8668740b471f4b0ad076ad3463e56be6b61 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 ddc687a45a0958160882c17eb2f15ce29c8d1524..da7f20233063eca2aa74e66cb34d285162698ce6 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 99828487f1902c310693fd64eabdd45bb515e9d0..68fef5234c9abe2609ded67820584a2ad4ff07fe 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 cb2083eb019bf7628759527a69a8f4ad09994e5c..4d8b01dbe6e1b11a96440f9dec921bd98fb874eb 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 6a3c41fb1155da78036dcafda2e4423cc2c70d3f..b2732de51058a414bbaaa9b64fc87b7b38d9d1ec 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 78b149b14b1d4bf0f7f8cdd4a1879ae634f18b27..3f42eba4ece0060c56ab56b803e0167fa7c2d1e3 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 0000000000000000000000000000000000000000..f07962096a32c36f6b114a8beac8c3f9c4cfef56
--- /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 0000000000000000000000000000000000000000..346f2b7856791a83ee6ba11a7dca70fddb7cedbc
--- /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 38a275d4389593e965df06a1c48df40250f69dc8..13d9dbdd9af2d7e1c2a7d898b04646620490a2ad 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 40e853c39ca9957a8d8f46963f533f660e636d7c..c6b65c7348ae0b6e9267631ce36fa2e9bd7c1577 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 ea936e815b24adf894da049466c7b9644d78dea5..b86923f07f793ecc847200045a65612e75f11a39 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 20c503d30c80101e63c218761c95e20044c3edeb..7a39d1af9e2d51821e8fbfb57e3c147d268aa463 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 0bac78d8a6bdf5978b69686ee08e74f4e2819ca8..6e7fd55fa4bb158b1b845ae230b6342957c32251 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 3bb936790d506f044a216ee3393bb9d09855e791..b543471a5d35b4f4dc7ae5472490d4fda870bdbe 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 bae3b37e267d5c9b2fde991fc4f9c31505a725a5..9f2924c23b73c7fbda060678b476e9dbd7707353 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 ad890b4e4d08e2f84190e4818d1936093575d2c5..c97543f57d8f32e23d1d79a079a3c2ba3f181dfe 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 25fe63c2655a83c2bcc2027189d88cb1ba618918..0000000000000000000000000000000000000000
--- 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 7c4f2b43618929ee8cfcf963f2ef63f830f9d933..a25ce35736146547bca4ec3f08218c9a96c60f30 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 6b0800af9c6d01c541149b87b0a316bd3cb39188..9274e01632d5864effa3bdc0aa0e4b3935b3bd96 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 73153d23c469894fded03734c718e606b23a0582..9af5d3a303ae25d7e6d0e5ddc24f10432558b0d5 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 1036b9f34e9dd9eac3eddd7ba6d38ba00163e34f..e83cd55e73691a44bbaa1b02a37876081cabe775 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 30415814adbbabee8048e258bb5749c3ca84e8b9..45c322427930d95adf1bfea36a16909a578f0801 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 8ca863e8b3667e3d9e65fca062ea799b71e06e8a..d8a3e859f85cdc65393d874046cd44d0e16ce84d 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 0000000000000000000000000000000000000000..67c0a434c9b52add95da62d2e1aae9b190242e6c
--- /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 a006224d5080c4ab09c6fb1935fc71dd02adb047..16ee5ec0f230f1fecf4f0f978485e909b0d46bde 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>