From c086bd5c6837a98d3c989c43f2b75aeaa0e5eff0 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 20 Mar 2014 12:43:16 -0700 Subject: [PATCH 01/18] Add HistoryServer and scripts ++ Refactor WebUI interface HistoryServer can be launched with ./sbin/start-history-server.sh and stopped with ./sbin/stop-history-server.sh. This commit also involves refactoring all the UIs to avoid duplicate code. --- .../deploy/{WebUI.scala => DeployWebUI.scala} | 2 +- .../spark/deploy/SparkUIContainer.scala | 50 ++++++ .../spark/deploy/history/HistoryServer.scala | 148 ++++++++++++++++++ .../history/HistoryServerArguments.scala | 81 ++++++++++ .../spark/deploy/history/IndexPage.scala | 71 +++++++++ .../apache/spark/deploy/master/Master.scala | 6 +- .../spark/deploy/master/ui/MasterWebUI.scala | 41 +---- .../spark/deploy/worker/ui/WorkerWebUI.scala | 16 +- .../scala/org/apache/spark/ui/SparkUI.scala | 35 ++--- .../scala/org/apache/spark/ui/WebUI.scala | 37 +++++ sbin/start-history-server.sh | 46 ++++++ sbin/stop-history-server.sh | 25 +++ 12 files changed, 490 insertions(+), 68 deletions(-) rename core/src/main/scala/org/apache/spark/deploy/{WebUI.scala => DeployWebUI.scala} (97%) create mode 100644 core/src/main/scala/org/apache/spark/deploy/SparkUIContainer.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala create mode 100644 core/src/main/scala/org/apache/spark/ui/WebUI.scala create mode 100755 sbin/start-history-server.sh create mode 100755 sbin/stop-history-server.sh diff --git a/core/src/main/scala/org/apache/spark/deploy/WebUI.scala b/core/src/main/scala/org/apache/spark/deploy/DeployWebUI.scala similarity index 97% rename from core/src/main/scala/org/apache/spark/deploy/WebUI.scala rename to core/src/main/scala/org/apache/spark/deploy/DeployWebUI.scala index ae258b58b9cc5..254f076f29441 100644 --- a/core/src/main/scala/org/apache/spark/deploy/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployWebUI.scala @@ -42,6 +42,6 @@ private[spark] object DeployWebUI { return "%.0f min".format(minutes) } val hours = minutes / 60 - return "%.1f h".format(hours) + "%.1f h".format(hours) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkUIContainer.scala b/core/src/main/scala/org/apache/spark/deploy/SparkUIContainer.scala new file mode 100644 index 0000000000000..33fceae4ff489 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/SparkUIContainer.scala @@ -0,0 +1,50 @@ +/* + * 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.deploy + +import org.apache.spark.ui.{SparkUI, WebUI} + +private[spark] abstract class SparkUIContainer(name: String) extends WebUI(name) { + + /** Attach a SparkUI to this container. Only valid after bind(). */ + def attachUI(ui: SparkUI) { + assert(serverInfo.isDefined, + "%s must be bound to a server before attaching SparkUIs".format(name)) + val rootHandler = serverInfo.get.rootHandler + for (handler <- ui.handlers) { + rootHandler.addHandler(handler) + if (!handler.isStarted) { + handler.start() + } + } + } + + /** Detach a SparkUI from this container. Only valid after bind(). */ + def detachUI(ui: SparkUI) { + assert(serverInfo.isDefined, + "%s must be bound to a server before detaching SparkUIs".format(name)) + val rootHandler = serverInfo.get.rootHandler + for (handler <- ui.handlers) { + if (handler.isStarted) { + handler.stop() + } + rootHandler.removeHandler(handler) + } + } + +} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala new file mode 100644 index 0000000000000..09f5d63b6c907 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -0,0 +1,148 @@ +/* + * 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.deploy.history + +import java.net.URI +import javax.servlet.http.HttpServletRequest + +import scala.collection.mutable + +import org.apache.hadoop.fs.{FileStatus, Path} +import org.eclipse.jetty.servlet.ServletContextHandler + +import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.deploy.SparkUIContainer +import org.apache.spark.ui.SparkUI +import org.apache.spark.ui.JettyUtils._ +import org.apache.spark.util.Utils +import org.apache.spark.scheduler.ReplayListenerBus + +/** + * A web server that re-renders SparkUIs of finished applications. + * + * For the standalone mode, MasterWebUI already achieves this functionality. Thus, the + * main use case of the HistoryServer is in other deploy modes (e.g. Yarn or Mesos). + * + * The logging directory structure is as follows: Within the given base directory, each + * application's event logs are maintained in the application's own sub-directory. + * + * @param baseLogDir The base directory in which event logs are found + * @param requestedPort The requested port to which this server is to be bound + */ +class HistoryServer(baseLogDir: String, requestedPort: Int, conf: SparkConf) + extends SparkUIContainer("History Server") with Logging { + + private val host = Utils.localHostName() + private val port = requestedPort + private val indexPage = new IndexPage(this) + private val fileSystem = Utils.getHadoopFileSystem(new URI(baseLogDir)) + private val securityManager = new SecurityManager(conf) + + private val handlers = Seq[ServletContextHandler]( + createStaticHandler(HistoryServer.STATIC_RESOURCE_DIR, "/static"), + createServletHandler("/", + (request: HttpServletRequest) => indexPage.render(request), securityMgr = securityManager) + ) + + // A mapping from an event log path to the associated, already rendered, SparkUI + val logPathToUI = mutable.HashMap[String, SparkUI]() + + // A mapping from an event log path to a timestamp of when it was last updated + val logPathToLastUpdated = mutable.HashMap[String, Long]() + + /** Bind to the HTTP server behind this web interface */ + override def bind() { + try { + serverInfo = Some(startJettyServer(host, port, handlers, conf)) + logInfo("Started HistoryServer at http://%s:%d".format(host, boundPort)) + } catch { + case e: Exception => + logError("Failed to create HistoryServer", e) + System.exit(1) + } + checkForLogs() + } + + /** + * Check for any updated event logs. + * + * If a new application is found, render the associated SparkUI and remember it. + * If an existing application is updated, re-render the associated SparkUI. + * If an existing application is removed, remove the associated SparkUI. + */ + def checkForLogs() { + val logStatus = fileSystem.listStatus(new Path(baseLogDir)) + val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]() + + // Render any missing or outdated SparkUI + logDirs.foreach { dir => + val path = dir.getPath.toString + val lastUpdated = dir.getModificationTime + if (!logPathToLastUpdated.contains(path) || + logPathToLastUpdated.getOrElse(path, -1L) < lastUpdated) { + maybeRenderUI(path, lastUpdated) + } + } + + // Remove any outdated SparkUIs + val logPaths = logDirs.map(_.getPath.toString) + logPathToUI.keys.foreach { path => + if (!logPaths.contains(path)) { + logPathToUI.remove(path) + logPathToLastUpdated.remove(path) + } + } + + logWarning("By the end of check for logs, the map looks like") + logPathToUI.foreach { case (k, v) => logWarning("* %s".format(k)) } + } + + /** Attempt to render a new SparkUI from event logs residing in the given log directory. */ + def maybeRenderUI(logPath: String, lastUpdated: Long) { + logWarning("Maybe rendering UI %s".format(logPath)) + + val appName = logPath.split("/").last + val replayBus = new ReplayListenerBus(conf) + val ui = new SparkUI(conf, replayBus, appName, "/history/%s".format(appName)) + + // Do not call ui.bind() to avoid creating a new server for each application + ui.start() + val success = replayBus.replay(logPath) + logWarning("Just replayed the events. Successful? %s".format(success)) + if (success) { + attachUI(ui) + logPathToUI(logPath) = ui + logPathToLastUpdated(logPath) = lastUpdated + } + } + +} + +object HistoryServer { + val STATIC_RESOURCE_DIR = SparkUI.STATIC_RESOURCE_DIR + + def main(argStrings: Array[String]) { + val conf = new SparkConf + val args = new HistoryServerArguments(argStrings, conf) + val server = new HistoryServer(args.logDir, args.port, conf) + server.bind() + + // Wait until the end of the world... or if the HistoryServer process is manually stopped + while(true) { Thread.sleep(Int.MaxValue) } + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala new file mode 100644 index 0000000000000..c142b18b94aea --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala @@ -0,0 +1,81 @@ +/* + * 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.deploy.history + +import java.net.URI + +import org.apache.spark.SparkConf +import org.apache.spark.util.{Utils, IntParam} +import org.apache.hadoop.fs.Path + +/** + * Command-line parser for the master. + */ +private[spark] class HistoryServerArguments(args: Array[String], conf: SparkConf) { + var port = 18080 + var logDir = "" + + parse(args.toList) + + def parse(args: List[String]): Unit = { + args match { + case ("--port" | "-p") :: IntParam(value) :: tail => + port = value + parse(tail) + + case ("--dir" | "-d") :: value :: tail => + logDir = value + parse(tail) + + case ("--help" | "-h") :: tail => + printUsageAndExit(0) + + case Nil => {} + + case _ => + printUsageAndExit(1) + } + validateLogDir() + } + + def validateLogDir() { + if (logDir == "") { + System.err.println("Logging directory must be specified.") + printUsageAndExit(1) + } + val fileSystem = Utils.getHadoopFileSystem(new URI(logDir)) + val path = new Path(logDir) + if (!fileSystem.exists(path) || !fileSystem.getFileStatus(path).isDir) { + System.err.println("Logging directory specified is invalid: %s".format(logDir)) + printUsageAndExit(1) + } + } + + /** + * Print usage and exit JVM with the given exit code. + */ + def printUsageAndExit(exitCode: Int) { + System.err.println( + "Usage: HistoryServer [options]\n" + + "\n" + + "Options:\n" + + " -p PORT, --port PORT Port for web server (default: 18080)\n" + + " -d DIR, --dir DIR Location of event log files") + System.exit(exitCode) + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala new file mode 100644 index 0000000000000..2200e41898942 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala @@ -0,0 +1,71 @@ +/* + * 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.deploy.history + +import javax.servlet.http.HttpServletRequest + +import scala.xml.Node + +import org.apache.spark.deploy.DeployWebUI +import org.apache.spark.deploy.master.ApplicationInfo +import org.apache.spark.ui.UIUtils +import org.apache.spark.util.Utils + +private[spark] class IndexPage(parent: HistoryServer) { + + def render(request: HttpServletRequest): Seq[Node] = { + val content = +
+
+
    +
  • + Welcome to the Fastest and Furious-est HistoryServer in the World! +
  • + { + parent.logPathToUI.map { case (path, ui) => +
  • {path} at {ui.basePath}
  • + } + } +
+
+
+ + UIUtils.basicSparkPage(content, "History Server") + } + + def appRow(app: ApplicationInfo): Seq[Node] = { + + + {app.id} + + + {app.desc.name} + + + {app.coresGranted} + + + {Utils.megabytesToString(app.desc.memoryPerSlave)} + + {DeployWebUI.formatDate(app.submitDate)} + {app.desc.user} + {app.state.toString} + {DeployWebUI.formatDuration(app.duration)} + + } +} 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 1fd211416976e..26ecfa406af5b 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 @@ -671,16 +671,16 @@ private[spark] class Master( appConf.set("spark.eventLog.compress", "true") appConf.set("spark.io.compression.codec", codec) } - val replayerBus = new ReplayListenerBus(appConf) + val replayBus = new ReplayListenerBus(appConf) val ui = new SparkUI( appConf, - replayerBus, + replayBus, "%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) + val success = replayBus.replay(eventLogDir) if (!success) { ui.stop() None 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 bd75b2dfd0e07..946eef782936b 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 @@ -22,8 +22,9 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.Logging +import org.apache.spark.deploy.SparkUIContainer import org.apache.spark.deploy.master.Master -import org.apache.spark.ui.{ServerInfo, SparkUI} +import org.apache.spark.ui.SparkUI import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.{AkkaUtils, Utils} @@ -31,7 +32,9 @@ import org.apache.spark.util.{AkkaUtils, Utils} * Web UI server for the standalone master. */ private[spark] -class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { +class MasterWebUI(val master: Master, requestedPort: Int) + extends SparkUIContainer("MasterWebUI") with Logging { + val masterActorRef = master.self val timeout = AkkaUtils.askTimeout(master.conf) @@ -39,7 +42,6 @@ class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { private val port = requestedPort private val applicationPage = new ApplicationPage(this) private val indexPage = new IndexPage(this) - private var serverInfo: Option[ServerInfo] = None private val handlers: Seq[ServletContextHandler] = { master.masterMetricsSystem.getServletHandlers ++ @@ -57,7 +59,8 @@ class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { ) } - def bind() { + /** Bind to the HTTP server behind this web interface. */ + override def bind() { try { serverInfo = Some(startJettyServer(host, port, handlers, master.conf)) logInfo("Started Master web UI at http://%s:%d".format(host, boundPort)) @@ -68,36 +71,6 @@ class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { } } - def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) - - /** 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() { - assert(serverInfo.isDefined, "Attempted to stop a Master UI that was not bound to a server!") - serverInfo.get.server.stop() - } } private[spark] object MasterWebUI { 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 de76a5d5eb7bc..335a891493a46 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 @@ -24,7 +24,7 @@ import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.Logging import org.apache.spark.deploy.worker.Worker -import org.apache.spark.ui.{JettyUtils, ServerInfo, SparkUI, UIUtils} +import org.apache.spark.ui.{SparkUI, UIUtils, WebUI} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.{AkkaUtils, Utils} @@ -33,7 +33,7 @@ import org.apache.spark.util.{AkkaUtils, Utils} */ private[spark] class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[Int] = None) - extends Logging { + extends WebUI("WorkerWebUI") with Logging { val timeout = AkkaUtils.askTimeout(worker.conf) @@ -41,7 +41,6 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I 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 ++ @@ -58,9 +57,10 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I ) } - def bind() { + /** Bind to the HTTP server behind this web interface. */ + override def bind() { try { - serverInfo = Some(JettyUtils.startJettyServer(host, port, handlers, worker.conf)) + serverInfo = Some(startJettyServer(host, port, handlers, worker.conf)) logInfo("Started Worker web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => @@ -69,8 +69,6 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I } } - def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) - private def log(request: HttpServletRequest): String = { val defaultBytes = 100 * 1024 @@ -187,10 +185,6 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I (startByte, endByte) } - def 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 { 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 fd638c83aac6e..e49d5b14722d8 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -36,7 +36,7 @@ private[spark] class SparkUI( val listenerBus: SparkListenerBus, val appName: String, val basePath: String = "") - extends Logging { + extends WebUI("SparkUI") with Logging { def this(sc: SparkContext) = this(sc, sc.conf, sc.listenerBus, sc.appName) def this(conf: SparkConf, listenerBus: SparkListenerBus, appName: String, basePath: String) = @@ -49,7 +49,6 @@ private[spark] class SparkUI( 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) @@ -76,20 +75,6 @@ private[spark] class SparkUI( // Maintain executor storage status through Spark events val storageStatusListener = new StorageStatusListener - /** Bind the HTTP server which backs this web interface */ - def bind() { - try { - 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) - System.exit(1) - } - } - - def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) - /** Initialize all components of the server */ def start() { storage.start() @@ -105,9 +90,21 @@ private[spark] class SparkUI( listenerBus.addListener(exec.listener) } - def stop() { - assert(serverInfo.isDefined, "Attempted to stop a SparkUI that was not bound to a server!") - serverInfo.get.server.stop() + /** Bind to the HTTP server behind this web interface. */ + override def bind() { + try { + 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) + System.exit(1) + } + } + + /** Stop the server behind this web interface. Only valid after bind(). */ + override def stop() { + super.stop() logInfo("Stopped Spark Web UI at %s".format(appUIAddress)) } diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala new file mode 100644 index 0000000000000..4026f1942d470 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -0,0 +1,37 @@ +/* + * 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.ui + +private[spark] abstract class WebUI(name: String) { + protected var serverInfo: Option[ServerInfo] = None + + /** + * Bind to the HTTP server behind this web interface. + * Overridden implementation should set serverInfo. + */ + def bind() { } + + /** Return the actual port to which this server is bound. Only valid after bind(). */ + def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) + + /** Stop the server behind this web interface. Only valid after bind(). */ + def stop() { + assert(serverInfo.isDefined, "Attempted to stop %s before binding to a server!".format(name)) + serverInfo.get.server.stop() + } +} diff --git a/sbin/start-history-server.sh b/sbin/start-history-server.sh new file mode 100755 index 0000000000000..76ca799862e00 --- /dev/null +++ b/sbin/start-history-server.sh @@ -0,0 +1,46 @@ +#!/usr/bin/env bash + +# +# 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. +# + +# Starts the history server on the machine this script is executed on. +# +# Usage: start-history-server.sh [] +# Example: ./start-history-server.sh --dir /tmp/spark-events --port 18080 +# + +sbin=`dirname "$0"` +sbin=`cd "$sbin"; pwd` + +if [ $# -lt 1 ]; then + echo "Usage: ./start-history-server.sh []" + echo "Example: ./start-history-server.sh /tmp/spark-events 18080" + exit +fi + +# Set up base event log directory +LOG_DIR=$1 +shift + +# Set up web UI port +if [ ! -z $1 ]; then + PORT=$1 +else + PORT=18080 +fi + +"$sbin"/spark-daemon.sh start org.apache.spark.deploy.history.HistoryServer 1 --dir "$LOG_DIR" --port "$PORT" diff --git a/sbin/stop-history-server.sh b/sbin/stop-history-server.sh new file mode 100755 index 0000000000000..c0034ad641cbe --- /dev/null +++ b/sbin/stop-history-server.sh @@ -0,0 +1,25 @@ +#!/usr/bin/env bash + +# +# 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. +# + +# Stops the history server on the machine this script is executed on. + +sbin=`dirname "$0"` +sbin=`cd "$sbin"; pwd` + +"$sbin"/spark-daemon.sh stop org.apache.spark.deploy.history.HistoryServer 1 From 8aac16355329809b11c76430fa8737d328f2e962 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 20 Mar 2014 14:34:34 -0700 Subject: [PATCH 02/18] Add basic application table --- .../spark/deploy/history/HistoryServer.scala | 21 +++---- .../spark/deploy/history/IndexPage.scala | 58 +++++++++---------- 2 files changed, 38 insertions(+), 41 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 09f5d63b6c907..623e136d8b7a3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -44,10 +44,10 @@ import org.apache.spark.scheduler.ReplayListenerBus * @param baseLogDir The base directory in which event logs are found * @param requestedPort The requested port to which this server is to be bound */ -class HistoryServer(baseLogDir: String, requestedPort: Int, conf: SparkConf) +class HistoryServer(val baseLogDir: String, requestedPort: Int, conf: SparkConf) extends SparkUIContainer("History Server") with Logging { - private val host = Utils.localHostName() + private val host = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(Utils.localHostName()) private val port = requestedPort private val indexPage = new IndexPage(this) private val fileSystem = Utils.getHadoopFileSystem(new URI(baseLogDir)) @@ -101,29 +101,24 @@ class HistoryServer(baseLogDir: String, requestedPort: Int, conf: SparkConf) // Remove any outdated SparkUIs val logPaths = logDirs.map(_.getPath.toString) - logPathToUI.keys.foreach { path => + logPathToUI.foreach { case (path, ui) => if (!logPaths.contains(path)) { + detachUI(ui) logPathToUI.remove(path) logPathToLastUpdated.remove(path) } } - - logWarning("By the end of check for logs, the map looks like") - logPathToUI.foreach { case (k, v) => logWarning("* %s".format(k)) } } /** Attempt to render a new SparkUI from event logs residing in the given log directory. */ def maybeRenderUI(logPath: String, lastUpdated: Long) { - logWarning("Maybe rendering UI %s".format(logPath)) - - val appName = logPath.split("/").last + val appName = getAppName(logPath) val replayBus = new ReplayListenerBus(conf) val ui = new SparkUI(conf, replayBus, appName, "/history/%s".format(appName)) // Do not call ui.bind() to avoid creating a new server for each application ui.start() val success = replayBus.replay(logPath) - logWarning("Just replayed the events. Successful? %s".format(success)) if (success) { attachUI(ui) logPathToUI(logPath) = ui @@ -131,6 +126,12 @@ class HistoryServer(baseLogDir: String, requestedPort: Int, conf: SparkConf) } } + /** Parse app name from the given log path. */ + def getAppName(logPath: String): String = logPath.split("/").last + + /** Return the address of this server. */ + def getAddress = "http://" + host + ":" + boundPort + } object HistoryServer { diff --git a/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala index 2200e41898942..acd8ff064d3bd 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala @@ -17,30 +17,36 @@ package org.apache.spark.deploy.history +import java.text.SimpleDateFormat +import java.util.Date import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.deploy.DeployWebUI -import org.apache.spark.deploy.master.ApplicationInfo -import org.apache.spark.ui.UIUtils -import org.apache.spark.util.Utils +import org.apache.spark.ui.{SparkUI, UIUtils} private[spark] class IndexPage(parent: HistoryServer) { + val dateFmt = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") def render(request: HttpServletRequest): Seq[Node] = { + // Check if logs have been updated + parent.checkForLogs() + + // Populate app table, with most recently modified first + val appRows = parent.logPathToLastUpdated.toSeq + .sortBy { case (path, lastUpdated) => -lastUpdated } + .map { case (path, lastUpdated) => + // (appName, lastUpdated, UI) + (parent.getAppName(path), lastUpdated, parent.logPathToUI(path)) + } + val appTable = UIUtils.listingTable(appHeader, appRow, appRows) + val content =
    -
  • - Welcome to the Fastest and Furious-est HistoryServer in the World! -
  • - { - parent.logPathToUI.map { case (path, ui) => -
  • {path} at {ui.basePath}
  • - } - } +
  • Event Log Location: {parent.baseLogDir}
  • +

    Applications

    {appTable}
@@ -48,24 +54,14 @@ private[spark] class IndexPage(parent: HistoryServer) { UIUtils.basicSparkPage(content, "History Server") } - def appRow(app: ApplicationInfo): Seq[Node] = { - - - {app.id} - - - {app.desc.name} - - - {app.coresGranted} - - - {Utils.megabytesToString(app.desc.memoryPerSlave)} - - {DeployWebUI.formatDate(app.submitDate)} - {app.desc.user} - {app.state.toString} - {DeployWebUI.formatDuration(app.duration)} - + private val appHeader = Seq[String]("App Name", "Last Updated") + + private def appRow(info: (String, Long, SparkUI)): Seq[Node] = { + info match { case (appName, lastUpdated, ui) => + + {appName} + {dateFmt.format(new Date(lastUpdated))} + + } } } From 758441890dc86c8ed069e6c684b21528038f2ff7 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 20 Mar 2014 21:59:34 -0700 Subject: [PATCH 03/18] Report application start/end times to HistoryServer This involves adding application start and end events. This also allows us to record the actual app name instead of simply using the name of the directory. --- .../scala/org/apache/spark/SparkContext.scala | 16 ++++ .../spark/deploy/history/HistoryServer.scala | 77 ++++++++++++------- .../spark/deploy/history/IndexPage.scala | 47 +++++++---- .../spark/scheduler/ApplicationListener.scala | 49 ++++++++++++ .../scheduler/EventLoggingListener.scala | 4 + .../spark/scheduler/LiveListenerBus.scala | 19 +++-- .../spark/scheduler/SparkListener.scala | 14 ++++ .../spark/scheduler/SparkListenerBus.scala | 4 + .../scala/org/apache/spark/ui/SparkUI.scala | 4 +- .../apache/spark/ui/env/EnvironmentUI.scala | 3 +- .../apache/spark/ui/exec/ExecutorsUI.scala | 3 +- .../org/apache/spark/ui/jobs/IndexPage.scala | 3 +- .../apache/spark/ui/jobs/JobProgressUI.scala | 3 +- .../org/apache/spark/ui/jobs/PoolPage.scala | 3 +- .../org/apache/spark/ui/jobs/StagePage.scala | 3 +- .../spark/ui/storage/BlockManagerUI.scala | 3 +- .../apache/spark/ui/storage/IndexPage.scala | 3 +- .../org/apache/spark/ui/storage/RDDPage.scala | 3 +- .../org/apache/spark/util/JsonProtocol.scala | 29 +++++++ .../apache/spark/util/JsonProtocolSuite.scala | 19 +++++ .../apache/spark/repl/SparkILoopInit.scala | 4 +- 21 files changed, 250 insertions(+), 63 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/scheduler/ApplicationListener.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index a1003b7925715..e9256c10b8e67 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -228,6 +228,7 @@ class SparkContext( dagScheduler.start() postEnvironmentUpdate() + postApplicationStart() /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ val hadoopConfiguration = { @@ -826,6 +827,7 @@ class SparkContext( /** Shut down the SparkContext. */ def stop() { + postApplicationEnd() ui.stop() eventLogger.foreach(_.stop()) // Do this only if not stopped already - best case effort. @@ -1066,6 +1068,20 @@ class SparkContext( /** Register a new RDD, returning its RDD ID */ private[spark] def newRddId(): Int = nextRddId.getAndIncrement() + /** Post the application start event */ + private def postApplicationStart() { + listenerBus.post(SparkListenerApplicationStart(appName, startTime)) + } + + /** + * Post the application end event to all listeners immediately, rather than adding it + * to the event queue for it to be asynchronously processed eventually. Otherwise, a race + * condition exists in which the listeners may stop before this event has been propagated. + */ + private def postApplicationEnd() { + listenerBus.post(SparkListenerApplicationEnd(System.currentTimeMillis), blocking = true) + } + /** Post the environment update event once the task scheduler is ready */ private def postEnvironmentUpdate() { if (taskScheduler != null) { diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 623e136d8b7a3..ea5875d7853bb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -30,7 +30,7 @@ import org.apache.spark.deploy.SparkUIContainer import org.apache.spark.ui.SparkUI import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.Utils -import org.apache.spark.scheduler.ReplayListenerBus +import org.apache.spark.scheduler.{ApplicationListener, ReplayListenerBus} /** * A web server that re-renders SparkUIs of finished applications. @@ -59,11 +59,7 @@ class HistoryServer(val baseLogDir: String, requestedPort: Int, conf: SparkConf) (request: HttpServletRequest) => indexPage.render(request), securityMgr = securityManager) ) - // A mapping from an event log path to the associated, already rendered, SparkUI - val logPathToUI = mutable.HashMap[String, SparkUI]() - - // A mapping from an event log path to a timestamp of when it was last updated - val logPathToLastUpdated = mutable.HashMap[String, Long]() + val appIdToInfo = mutable.HashMap[String, ApplicationHistoryInfo]() /** Bind to the HTTP server behind this web interface */ override def bind() { @@ -78,6 +74,12 @@ class HistoryServer(val baseLogDir: String, requestedPort: Int, conf: SparkConf) checkForLogs() } + /** Parse app ID from the given log path. */ + def getAppId(logPath: String): String = logPath.split("/").last + + /** Return the address of this server. */ + def getAddress = "http://" + host + ":" + boundPort + /** * Check for any updated event logs. * @@ -92,46 +94,56 @@ class HistoryServer(val baseLogDir: String, requestedPort: Int, conf: SparkConf) // Render any missing or outdated SparkUI logDirs.foreach { dir => val path = dir.getPath.toString - val lastUpdated = dir.getModificationTime - if (!logPathToLastUpdated.contains(path) || - logPathToLastUpdated.getOrElse(path, -1L) < lastUpdated) { - maybeRenderUI(path, lastUpdated) + val appId = getAppId(path) + val lastUpdated = { + val logFiles = fileSystem.listStatus(dir.getPath) + if (logFiles != null) logFiles.map(_.getModificationTime).max else dir.getModificationTime + } + if (!appIdToInfo.contains(appId) || appIdToInfo(appId).lastUpdated < lastUpdated) { + maybeRenderUI(appId, path, lastUpdated) } } // Remove any outdated SparkUIs - val logPaths = logDirs.map(_.getPath.toString) - logPathToUI.foreach { case (path, ui) => - if (!logPaths.contains(path)) { - detachUI(ui) - logPathToUI.remove(path) - logPathToLastUpdated.remove(path) + val appIds = logDirs.map { dir => getAppId(dir.getPath.toString) } + appIdToInfo.foreach { case (appId, info) => + if (!appIds.contains(appId)) { + detachUI(info.ui) + appIdToInfo.remove(appId) } } } /** Attempt to render a new SparkUI from event logs residing in the given log directory. */ - def maybeRenderUI(logPath: String, lastUpdated: Long) { - val appName = getAppName(logPath) + private def maybeRenderUI(appId: String, logPath: String, lastUpdated: Long) { val replayBus = new ReplayListenerBus(conf) - val ui = new SparkUI(conf, replayBus, appName, "/history/%s".format(appName)) + val appListener = new ApplicationListener + replayBus.addListener(appListener) + val ui = new SparkUI(conf, replayBus, appId, "/history/%s".format(appId)) // Do not call ui.bind() to avoid creating a new server for each application ui.start() val success = replayBus.replay(logPath) if (success) { attachUI(ui) - logPathToUI(logPath) = ui - logPathToLastUpdated(logPath) = lastUpdated + if (!appListener.started) { + logWarning("Application has event logs but has not started: %s".format(appId)) + } + val appName = appListener.appName + val startTime = appListener.startTime + val endTime = appListener.endTime + val info = ApplicationHistoryInfo(appName, startTime, endTime, lastUpdated, logPath, ui) + + // If the UI already exists, terminate it and replace it + appIdToInfo.remove(appId).foreach { info => detachUI(info.ui) } + appIdToInfo(appId) = info + + // Use mnemonic original app name rather than app ID + val originalAppName = "%s (history)".format(appName) + ui.setAppName(originalAppName) } } - /** Parse app name from the given log path. */ - def getAppName(logPath: String): String = logPath.split("/").last - - /** Return the address of this server. */ - def getAddress = "http://" + host + ":" + boundPort - } object HistoryServer { @@ -147,3 +159,14 @@ object HistoryServer { while(true) { Thread.sleep(Int.MaxValue) } } } + +private[spark] case class ApplicationHistoryInfo( + name: String, + startTime: Long, + endTime: Long, + lastUpdated: Long, + logPath: String, + ui: SparkUI) { + def started = startTime != -1 + def finished = endTime != -1 +} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala index acd8ff064d3bd..a92cf3160dcfc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala @@ -23,22 +23,18 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.ui.{SparkUI, UIUtils} +import org.apache.spark.deploy.DeployWebUI +import org.apache.spark.ui.UIUtils private[spark] class IndexPage(parent: HistoryServer) { - val dateFmt = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + private val dateFmt = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") def render(request: HttpServletRequest): Seq[Node] = { // Check if logs have been updated parent.checkForLogs() - // Populate app table, with most recently modified first - val appRows = parent.logPathToLastUpdated.toSeq - .sortBy { case (path, lastUpdated) => -lastUpdated } - .map { case (path, lastUpdated) => - // (appName, lastUpdated, UI) - (parent.getAppName(path), lastUpdated, parent.logPathToUI(path)) - } + // Populate app table, with most recently modified app first + val appRows = parent.appIdToInfo.values.toSeq.sortBy { app => -app.lastUpdated } val appTable = UIUtils.listingTable(appHeader, appRow, appRows) val content = @@ -54,14 +50,31 @@ private[spark] class IndexPage(parent: HistoryServer) { UIUtils.basicSparkPage(content, "History Server") } - private val appHeader = Seq[String]("App Name", "Last Updated") + private val appHeader = Seq( + "App Name", + "Started", + "Finished", + "Duration", + "Log Directory", + "Last Updated") - private def appRow(info: (String, Long, SparkUI)): Seq[Node] = { - info match { case (appName, lastUpdated, ui) => - - {appName} - {dateFmt.format(new Date(lastUpdated))} - - } + private def appRow(info: ApplicationHistoryInfo): Seq[Node] = { + val appName = if (info.started) info.name else parent.getAppId(info.logPath) + val uiAddress = parent.getAddress + info.ui.basePath + val startTime = if (info.started) dateFmt.format(new Date(info.startTime)) else "Not started" + val endTime = if (info.finished) dateFmt.format(new Date(info.endTime)) else "Not finished" + val difference = if (info.started && info.finished) info.endTime - info.startTime else -1L + val duration = if (difference > 0) DeployWebUI.formatDuration(difference) else "---" + val logDirectory = parent.getAppId(info.logPath) + val lastUpdated = dateFmt.format(new Date(info.lastUpdated)) + + + {appName} + {startTime} + {endTime} + {duration} + {logDirectory} + {lastUpdated} + } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ApplicationListener.scala b/core/src/main/scala/org/apache/spark/scheduler/ApplicationListener.scala new file mode 100644 index 0000000000000..ef984ed23dcff --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/ApplicationListener.scala @@ -0,0 +1,49 @@ +/* + * 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 + +/** + * A simple listener for application events. + * + * This listener assumes at most one of each of SparkListenerApplicationStart and + * SparkListenerApplicationEnd will be received. Otherwise, only the latest event + * of each type will take effect. + */ +private[spark] class ApplicationListener extends SparkListener { + var appName = "" + var startTime = -1L + var endTime = -1L + + def started = startTime != -1 + + def finished = endTime != -1 + + def duration: Long = { + val difference = endTime - startTime + if (started && finished && difference > 0) difference else -1L + } + + override def onApplicationStart(applicationStart: SparkListenerApplicationStart) { + appName = applicationStart.appName + startTime = applicationStart.time + } + + override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) { + endTime = applicationEnd.time + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 217f8825c2ae9..98cbdcb18ce2a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -90,6 +90,10 @@ private[spark] class EventLoggingListener(appName: String, conf: SparkConf) logEvent(event, flushLogger = true) override def onUnpersistRDD(event: SparkListenerUnpersistRDD) = logEvent(event, flushLogger = true) + override def onApplicationStart(event: SparkListenerApplicationStart) = + logEvent(event, flushLogger = true) + override def onApplicationEnd(event: SparkListenerApplicationEnd) = + logEvent(event, flushLogger = true) def stop() = logger.stop() } diff --git a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala index 353a48661b0f7..ebc6eb8fb0fd4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala @@ -64,13 +64,18 @@ private[spark] class LiveListenerBus extends SparkListenerBus with Logging { }.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 + def post(event: SparkListenerEvent, blocking: Boolean = false) { + if (!blocking) { + 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 + } + } else { + // Bypass the event queue and post to all attached listeners immediately + postToAll(event) } } 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 d4eb0ac88d8e8..5ca06abd1b5fc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -62,6 +62,10 @@ case class SparkListenerBlockManagerRemoved(blockManagerId: BlockManagerId) case class SparkListenerUnpersistRDD(rddId: Int) extends SparkListenerEvent +case class SparkListenerApplicationStart(appName: String, time: Long) extends SparkListenerEvent + +case class SparkListenerApplicationEnd(time: Long) extends SparkListenerEvent + /** An event used in the listener to shutdown the listener daemon thread. */ private[spark] case object SparkListenerShutdown extends SparkListenerEvent @@ -125,6 +129,16 @@ trait SparkListener { * Called when an RDD is manually unpersisted by the application */ def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD) { } + + /** + * Called when the application starts + */ + def onApplicationStart(applicationStart: SparkListenerApplicationStart) { } + + /** + * Called when the application ends + */ + def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) { } } /** 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 729e120497571..d6df193d9bcf8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -61,6 +61,10 @@ private[spark] trait SparkListenerBus { sparkListeners.foreach(_.onBlockManagerRemoved(blockManagerRemoved)) case unpersistRDD: SparkListenerUnpersistRDD => sparkListeners.foreach(_.onUnpersistRDD(unpersistRDD)) + case applicationStart: SparkListenerApplicationStart => + sparkListeners.foreach(_.onApplicationStart(applicationStart)) + case applicationEnd: SparkListenerApplicationEnd => + sparkListeners.foreach(_.onApplicationEnd(applicationEnd)) case SparkListenerShutdown => } } 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 e49d5b14722d8..dc1b0c986c1d7 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -34,7 +34,7 @@ private[spark] class SparkUI( val sc: SparkContext, conf: SparkConf, val listenerBus: SparkListenerBus, - val appName: String, + var appName: String, val basePath: String = "") extends WebUI("SparkUI") with Logging { @@ -75,6 +75,8 @@ private[spark] class SparkUI( // Maintain executor storage status through Spark events val storageStatusListener = new StorageStatusListener + def setAppName(name: String) = appName = name + /** Initialize all components of the server */ def start() { storage.start() 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 23e90c34d5b33..33df97187ea78 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 @@ -29,10 +29,11 @@ import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.Page.Environment private[ui] class EnvironmentUI(parent: SparkUI) { - private val appName = parent.appName private val basePath = parent.basePath private var _listener: Option[EnvironmentListener] = None + private def appName = parent.appName + lazy val listener = _listener.get def start() { 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 031ed88a493a8..77a38a1d3aa7c 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 @@ -33,10 +33,11 @@ import org.apache.spark.ui.{SparkUI, UIUtils} import org.apache.spark.util.Utils private[ui] class ExecutorsUI(parent: SparkUI) { - private val appName = parent.appName private val basePath = parent.basePath private var _listener: Option[ExecutorsListener] = None + private def appName = parent.appName + lazy val listener = _listener.get def start() { 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 f3c93d4214ad0..b48be85899cc2 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 @@ -27,13 +27,14 @@ import org.apache.spark.ui.UIUtils /** Page showing list of all ongoing and recently finished stages and pools*/ 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 + private def appName = parent.appName + def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { val activeStages = listener.activeStages.values.toSeq 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 ee4e9c69c1bd1..f4c7b56824759 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 @@ -30,7 +30,6 @@ import org.apache.spark.util.Utils /** Web UI showing progress status of all jobs in the given SparkContext. */ 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 @@ -44,6 +43,8 @@ private[ui] class JobProgressUI(parent: SparkUI) { private val poolPage = new PoolPage(this) private var _listener: Option[JobProgressListener] = None + def appName = parent.appName + def start() { val conf = if (live) sc.conf else new SparkConf _listener = Some(new JobProgressListener(conf)) 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 bd33182b70059..3638e6035ba81 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 @@ -27,12 +27,13 @@ import org.apache.spark.ui.UIUtils /** Page showing specific pool details */ 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 + private def appName = parent.appName + def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { val poolName = request.getParameter("poolname") 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 da7f20233063e..0387546a1de96 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 @@ -28,11 +28,12 @@ import org.apache.spark.util.{Utils, Distribution} /** Page showing statistics and task list for a given stage */ 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 + private def appName = parent.appName + def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { val stageId = request.getParameter("id").toInt 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 4d8b01dbe6e1b..36d3390ec3028 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 @@ -30,7 +30,6 @@ import org.apache.spark.storage.{RDDInfo, StorageStatusListener, StorageUtils} /** Web UI showing storage status of all RDD's in the given SparkContext. */ private[ui] class BlockManagerUI(parent: SparkUI) { - val appName = parent.appName val basePath = parent.basePath private val indexPage = new IndexPage(this) @@ -39,6 +38,8 @@ private[ui] class BlockManagerUI(parent: SparkUI) { lazy val listener = _listener.get + def appName = parent.appName + def start() { _listener = Some(new BlockManagerListener(parent.storageStatusListener)) } 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 b2732de51058a..a657df082fe89 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 @@ -28,10 +28,11 @@ import org.apache.spark.util.Utils /** Page showing list of RDD's currently stored in the cluster */ private[ui] class IndexPage(parent: BlockManagerUI) { - private val appName = parent.appName private val basePath = parent.basePath private lazy val listener = parent.listener + private def appName = parent.appName + def render(request: HttpServletRequest): Seq[Node] = { val rdds = listener.rddInfoList val content = UIUtils.listingTable(rddHeader, rddRow, rdds) 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 3f42eba4ece00..75ee9976d7b5f 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 @@ -28,10 +28,11 @@ import org.apache.spark.util.Utils /** Page showing storage details for a given RDD */ private[ui] class RDDPage(parent: BlockManagerUI) { - private val appName = parent.appName private val basePath = parent.basePath private lazy val listener = parent.listener + private def appName = parent.appName + def render(request: HttpServletRequest): Seq[Node] = { val rddId = request.getParameter("id").toInt val storageStatusList = listener.storageStatusList diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 346f2b7856791..088b880ceaeb5 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -62,6 +62,10 @@ private[spark] object JsonProtocol { blockManagerRemovedToJson(blockManagerRemoved) case unpersistRDD: SparkListenerUnpersistRDD => unpersistRDDToJson(unpersistRDD) + case applicationStart: SparkListenerApplicationStart => + applicationStartToJson(applicationStart) + case applicationEnd: SparkListenerApplicationEnd => + applicationEndToJson(applicationEnd) // Not used, but keeps compiler happy case SparkListenerShutdown => JNothing @@ -157,6 +161,17 @@ private[spark] object JsonProtocol { ("RDD ID" -> unpersistRDD.rddId) } + def applicationStartToJson(applicationStart: SparkListenerApplicationStart): JValue = { + ("Event" -> Utils.getFormattedClassName(applicationStart)) ~ + ("App Name" -> applicationStart.appName) ~ + ("Timestamp" -> applicationStart.time) + } + + def applicationEndToJson(applicationEnd: SparkListenerApplicationEnd): JValue = { + ("Event" -> Utils.getFormattedClassName(applicationEnd)) ~ + ("Timestamp" -> applicationEnd.time) + } + /** ------------------------------------------------------------------- * * JSON serialization methods for classes SparkListenerEvents depend on | @@ -372,6 +387,8 @@ private[spark] object JsonProtocol { val blockManagerAdded = Utils.getFormattedClassName(SparkListenerBlockManagerAdded) val blockManagerRemoved = Utils.getFormattedClassName(SparkListenerBlockManagerRemoved) val unpersistRDD = Utils.getFormattedClassName(SparkListenerUnpersistRDD) + val applicationStart = Utils.getFormattedClassName(SparkListenerApplicationStart) + val applicationEnd = Utils.getFormattedClassName(SparkListenerApplicationEnd) (json \ "Event").extract[String] match { case `stageSubmitted` => stageSubmittedFromJson(json) @@ -385,6 +402,8 @@ private[spark] object JsonProtocol { case `blockManagerAdded` => blockManagerAddedFromJson(json) case `blockManagerRemoved` => blockManagerRemovedFromJson(json) case `unpersistRDD` => unpersistRDDFromJson(json) + case `applicationStart` => applicationStartFromJson(json) + case `applicationEnd` => applicationEndFromJson(json) } } @@ -456,6 +475,16 @@ private[spark] object JsonProtocol { SparkListenerUnpersistRDD((json \ "RDD ID").extract[Int]) } + def applicationStartFromJson(json: JValue): SparkListenerApplicationStart = { + val appName = (json \ "App Name").extract[String] + val time = (json \ "Timestamp").extract[Long] + SparkListenerApplicationStart(appName, time) + } + + def applicationEndFromJson(json: JValue): SparkListenerApplicationEnd = { + SparkListenerApplicationEnd((json \ "Timestamp").extract[Long]) + } + /** --------------------------------------------------------------------- * * JSON deserialization methods for classes SparkListenerEvents depend on | diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 67c0a434c9b52..54c32f0821dfc 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -52,6 +52,8 @@ class JsonProtocolSuite extends FunSuite { val blockManagerRemoved = SparkListenerBlockManagerRemoved( BlockManagerId("Scarce", "to be counted...", 100, 200)) val unpersistRdd = SparkListenerUnpersistRDD(12345) + val applicationStart = SparkListenerApplicationStart("The winner of all", 42L) + val applicationEnd = SparkListenerApplicationEnd(42L) testEvent(stageSubmitted, stageSubmittedJsonString) testEvent(stageCompleted, stageCompletedJsonString) @@ -64,6 +66,8 @@ class JsonProtocolSuite extends FunSuite { testEvent(blockManagerAdded, blockManagerAddedJsonString) testEvent(blockManagerRemoved, blockManagerRemovedJsonString) testEvent(unpersistRdd, unpersistRDDJsonString) + testEvent(applicationStart, applicationStartJsonString) + testEvent(applicationEnd, applicationEndJsonString) } test("Dependent Classes") { @@ -211,6 +215,11 @@ class JsonProtocolSuite extends FunSuite { assertEquals(e1.blockManagerId, e2.blockManagerId) case (e1: SparkListenerUnpersistRDD, e2: SparkListenerUnpersistRDD) => assert(e1.rddId == e2.rddId) + case (e1: SparkListenerApplicationStart, e2: SparkListenerApplicationStart) => + assert(e1.appName == e2.appName) + assert(e1.time == e2.time) + case (e1: SparkListenerApplicationEnd, e2: SparkListenerApplicationEnd) => + assert(e1.time == e2.time) case (SparkListenerShutdown, SparkListenerShutdown) => case _ => fail("Events don't match in types!") } @@ -556,4 +565,14 @@ class JsonProtocolSuite extends FunSuite { {"Event":"SparkListenerUnpersistRDD","RDD ID":12345} """ + private val applicationStartJsonString = + """ + {"Event":"SparkListenerApplicationStart","App Name":"The winner of all","Timestamp":42} + """ + + private val applicationEndJsonString = + """ + {"Event":"SparkListenerApplicationEnd","Timestamp":42} + """ + } diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala index 3ebf288130fb6..910b31d209e13 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala @@ -116,14 +116,14 @@ trait SparkILoopInit { } } - def initializeSpark() { + def initializeSpark() { intp.beQuietDuring { command(""" @transient val sc = org.apache.spark.repl.Main.interp.createSparkContext(); """) command("import org.apache.spark.SparkContext._") } - echo("Spark context available as sc.") + echo("Spark context available as sc.") } // code to be executed only after the interpreter is initialized From 60bc6d57577742e861d62c183ec56d9893e3ea6a Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 21 Mar 2014 18:17:43 -0700 Subject: [PATCH 04/18] First complete implementation of HistoryServer (only for finished apps) This involves a change in Spark's event log format. All event logs are now prefixed with EVENT_LOG_. If compression is used, the logger creates a special empty file prefixed with COMPRESSION_CODEC_ that indicates which codec is used. After the application finishes, the logger logs a special empty file named APPLICATION_COMPLETE. The ReplayListenerBus is now responsible for parsing all of the above file formats. In this commit, we establish a one-to-one mapping between ReplayListenerBus and event logging applications. The semantics of the ReplayListenerBus is further clarified (e.g. replay is not allowed before starting, and can only be called once). This commit also adds a control mechanism for the frequency at which HistoryServer accesses the disk to check for log updates. This enforces a minimum interval of N seconds between two checks, where N is arbitrarily chosen to be 5. --- .../scala/org/apache/spark/SparkContext.scala | 5 +- .../spark/deploy/ApplicationDescription.scala | 4 +- .../spark/deploy/history/HistoryServer.scala | 138 +++++++++++------- .../spark/deploy/history/IndexPage.scala | 6 +- .../apache/spark/deploy/master/Master.scala | 30 +--- .../spark/scheduler/ApplicationListener.scala | 8 +- .../scheduler/EventLoggingListener.scala | 55 +++++-- .../spark/scheduler/ReplayListenerBus.scala | 106 ++++++++++---- .../cluster/SparkDeploySchedulerBackend.scala | 3 +- .../apache/spark/storage/FileSegment.scala | 2 +- .../scala/org/apache/spark/ui/SparkUI.scala | 4 +- .../org/apache/spark/util/FileLogger.scala | 27 ++-- 12 files changed, 237 insertions(+), 151 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index e9256c10b8e67..b0cac873ba24d 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -164,15 +164,12 @@ class SparkContext( private[spark] val eventLogger: Option[EventLoggingListener] = { if (conf.getBoolean("spark.eventLog.enabled", false)) { val logger = new EventLoggingListener(appName, conf) + logger.start() 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() 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 15fa8a7679874..86305d2ea8a09 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala @@ -17,8 +17,6 @@ package org.apache.spark.deploy -import org.apache.spark.scheduler.EventLoggingInfo - private[spark] class ApplicationDescription( val name: String, val maxCores: Option[Int], @@ -26,7 +24,7 @@ private[spark] class ApplicationDescription( val command: Command, val sparkHome: Option[String], var appUiUrl: String, - val eventLogInfo: Option[EventLoggingInfo] = None) + val eventLogDir: Option[String] = None) extends Serializable { val user = System.getProperty("user.name", "") diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index ea5875d7853bb..b36abbd9d4ff4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -53,6 +53,9 @@ class HistoryServer(val baseLogDir: String, requestedPort: Int, conf: SparkConf) private val fileSystem = Utils.getHadoopFileSystem(new URI(baseLogDir)) private val securityManager = new SecurityManager(conf) + // A timestamp of when the disk was last accessed to check for log updates + private var lastLogCheck = -1L + private val handlers = Seq[ServletContextHandler]( createStaticHandler(HistoryServer.STATIC_RESOURCE_DIR, "/static"), createServletHandler("/", @@ -74,81 +77,106 @@ class HistoryServer(val baseLogDir: String, requestedPort: Int, conf: SparkConf) checkForLogs() } - /** Parse app ID from the given log path. */ - def getAppId(logPath: String): String = logPath.split("/").last - - /** Return the address of this server. */ - def getAddress = "http://" + host + ":" + boundPort - /** - * Check for any updated event logs. + * Check for any updates to event logs in the base directory. + * + * If a new finished application is found, the server renders the associated SparkUI + * from the application's event logs, attaches this UI to itself, and stores metadata + * information for this application. * - * If a new application is found, render the associated SparkUI and remember it. - * If an existing application is updated, re-render the associated SparkUI. - * If an existing application is removed, remove the associated SparkUI. + * If the logs for an existing finished application are no longer found, remove all + * associated information and detach the SparkUI. */ def checkForLogs() { - val logStatus = fileSystem.listStatus(new Path(baseLogDir)) - val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]() - - // Render any missing or outdated SparkUI - logDirs.foreach { dir => - val path = dir.getPath.toString - val appId = getAppId(path) - val lastUpdated = { - val logFiles = fileSystem.listStatus(dir.getPath) - if (logFiles != null) logFiles.map(_.getModificationTime).max else dir.getModificationTime - } - if (!appIdToInfo.contains(appId) || appIdToInfo(appId).lastUpdated < lastUpdated) { - maybeRenderUI(appId, path, lastUpdated) + if (logCheckReady) { + lastLogCheck = System.currentTimeMillis + val logStatus = fileSystem.listStatus(new Path(baseLogDir)) + val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]() + + // Render SparkUI for any new completed applications + logDirs.foreach { dir => + val path = dir.getPath.toString + val appId = getAppId(path) + val lastUpdated = getModificationTime(dir) + if (!appIdToInfo.contains(appId)) { + maybeRenderUI(appId, path, lastUpdated) + } } - } - // Remove any outdated SparkUIs - val appIds = logDirs.map { dir => getAppId(dir.getPath.toString) } - appIdToInfo.foreach { case (appId, info) => - if (!appIds.contains(appId)) { - detachUI(info.ui) - appIdToInfo.remove(appId) + // Remove any outdated SparkUIs + val appIds = logDirs.map { dir => getAppId(dir.getPath.toString) } + appIdToInfo.foreach { case (appId, info) => + if (!appIds.contains(appId)) { + detachUI(info.ui) + appIdToInfo.remove(appId) + } } } } - /** Attempt to render a new SparkUI from event logs residing in the given log directory. */ + /** + * Render a new SparkUI from the event logs if the associated application is finished. + * + * HistoryServer looks for a special file that indicates application completion in the given + * directory. If this file exists, the associated application is regarded to be complete, in + * which case the server proceeds to render the SparkUI. Otherwise, the server does nothing. + */ private def maybeRenderUI(appId: String, logPath: String, lastUpdated: Long) { - val replayBus = new ReplayListenerBus(conf) - val appListener = new ApplicationListener - replayBus.addListener(appListener) - val ui = new SparkUI(conf, replayBus, appId, "/history/%s".format(appId)) - - // Do not call ui.bind() to avoid creating a new server for each application - ui.start() - val success = replayBus.replay(logPath) - if (success) { - attachUI(ui) - if (!appListener.started) { - logWarning("Application has event logs but has not started: %s".format(appId)) + val replayBus = new ReplayListenerBus(logPath) + replayBus.start() + + // If the application completion file is found + if (replayBus.isApplicationComplete) { + val ui = new SparkUI(replayBus, appId, "/history/%s".format(appId)) + val appListener = new ApplicationListener + replayBus.addListener(appListener) + + // Do not call ui.bind() to avoid creating a new server for each application + ui.start() + val success = replayBus.replay() + if (success) { + attachUI(ui) + val appName = if (appListener.applicationStarted) appListener.appName else appId + ui.setAppName("%s (history)".format(appName)) + val startTime = appListener.startTime + val endTime = appListener.endTime + val info = ApplicationHistoryInfo(appName, startTime, endTime, lastUpdated, logPath, ui) + appIdToInfo(appId) = info } - val appName = appListener.appName - val startTime = appListener.startTime - val endTime = appListener.endTime - val info = ApplicationHistoryInfo(appName, startTime, endTime, lastUpdated, logPath, ui) - - // If the UI already exists, terminate it and replace it - appIdToInfo.remove(appId).foreach { info => detachUI(info.ui) } - appIdToInfo(appId) = info - - // Use mnemonic original app name rather than app ID - val originalAppName = "%s (history)".format(appName) - ui.setAppName(originalAppName) + } else { + logWarning("Skipping incomplete application: %s".format(logPath)) + } + replayBus.stop() + } + + /** Parse app ID from the given log path. */ + def getAppId(logPath: String): String = logPath.split("/").last + + /** Return the address of this server. */ + def getAddress = "http://" + host + ":" + boundPort + + /** Return when this directory is last modified. */ + private def getModificationTime(dir: FileStatus): Long = { + val logFiles = fileSystem.listStatus(dir.getPath) + if (logFiles != null) { + logFiles.map(_.getModificationTime).max + } else { + dir.getModificationTime } } + /** Return whether the last log check has happened sufficiently long ago. */ + private def logCheckReady: Boolean = { + System.currentTimeMillis - lastLogCheck > HistoryServer.UPDATE_INTERVAL_SECONDS * 1000 + } } object HistoryServer { val STATIC_RESOURCE_DIR = SparkUI.STATIC_RESOURCE_DIR + // Minimum interval between each check for logs, which requires a disk access + val UPDATE_INTERVAL_SECONDS = 5 + def main(argStrings: Array[String]) { val conf = new SparkConf val args = new HistoryServerArguments(argStrings, conf) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala index a92cf3160dcfc..2b997026b3a47 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala @@ -30,19 +30,18 @@ private[spark] class IndexPage(parent: HistoryServer) { private val dateFmt = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") def render(request: HttpServletRequest): Seq[Node] = { - // Check if logs have been updated parent.checkForLogs() // Populate app table, with most recently modified app first val appRows = parent.appIdToInfo.values.toSeq.sortBy { app => -app.lastUpdated } val appTable = UIUtils.listingTable(appHeader, appRow, appRows) - val content =
  • Event Log Location: {parent.baseLogDir}
  • -

    Applications

    {appTable} +

    +

    Finished Applications

    {appTable}
@@ -67,7 +66,6 @@ private[spark] class IndexPage(parent: HistoryServer) { val duration = if (difference > 0) DeployWebUI.formatDuration(difference) else "---" val logDirectory = parent.getAppId(info.logPath) val lastUpdated = dateFmt.format(new Date(info.lastUpdated)) - {appName} {startTime} 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 26ecfa406af5b..ad15de5e4ce07 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 @@ -149,7 +149,6 @@ private[spark] class Master( override def postStop() { webUi.stop() - appIdToUI.values.foreach(_.stop()) masterMetricsSystem.stop() applicationMetricsSystem.stop() persistenceEngine.close() @@ -622,10 +621,7 @@ private[spark] class Master( 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) - } + appIdToUI.remove(a.id).foreach { ui => webUi.detachUI(ui) } applicationMetricsSystem.removeSource(a.appSource) }) completedApps.trimStart(toRemove) @@ -663,28 +659,14 @@ private[spark] class Master( */ 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 replayBus = new ReplayListenerBus(appConf) - val ui = new SparkUI( - appConf, - replayBus, - "%s (finished)".format(appName), - "/history/%s".format(app.id)) + val eventLogDir = app.desc.eventLogDir.getOrElse { return None } + val replayBus = new ReplayListenerBus(eventLogDir) + val ui = new SparkUI(replayBus, "%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 = replayBus.replay(eventLogDir) - if (!success) { - ui.stop() - None - } else Some(ui) + val success = replayBus.replay() + if (success) Some(ui) else None } /** Generate a new app ID given a app's submission date */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/ApplicationListener.scala b/core/src/main/scala/org/apache/spark/scheduler/ApplicationListener.scala index ef984ed23dcff..9a20ad1bb5ef4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ApplicationListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ApplicationListener.scala @@ -29,13 +29,13 @@ private[spark] class ApplicationListener extends SparkListener { var startTime = -1L var endTime = -1L - def started = startTime != -1 + def applicationStarted = startTime != -1 - def finished = endTime != -1 + def applicationFinished = endTime != -1 - def duration: Long = { + def applicationDuration: Long = { val difference = endTime - startTime - if (started && finished && difference > 0) difference else -1L + if (applicationStarted && applicationFinished && difference > 0) difference else -1L } override def onApplicationStart(applicationStart: SparkListenerApplicationStart) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 98cbdcb18ce2a..ead6a2904228b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -36,6 +36,8 @@ import org.apache.spark.util.{JsonProtocol, FileLogger} private[spark] class EventLoggingListener(appName: String, conf: SparkConf) extends SparkListener with Logging { + import EventLoggingListener._ + 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 @@ -46,16 +48,19 @@ private[spark] class EventLoggingListener(appName: String, conf: SparkConf) 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) + /** + * Begin logging events. If compression is used, log a file that indicates which compression + * library is used. + */ + def start() { + logInfo("Logging events to %s".format(logDir)) + if (shouldCompress) { + val codec = conf.get("spark.io.compression.codec", CompressionCodec.DEFAULT_COMPRESSION_CODEC) + logger.newFile(COMPRESSION_CODEC_PREFIX + codec) + } + logger.newFile(LOG_PREFIX + logger.fileIndex) } - 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))) @@ -95,8 +100,36 @@ private[spark] class EventLoggingListener(appName: String, conf: SparkConf) override def onApplicationEnd(event: SparkListenerApplicationEnd) = logEvent(event, flushLogger = true) - def stop() = logger.stop() + /** + * Stop logging events. In addition, create an empty special file to indicate application + * completion. + */ + def stop() = { + logger.newFile(APPLICATION_COMPLETE) + logger.stop() + } } -// If compression is not enabled, compressionCodec is None -private[spark] case class EventLoggingInfo(logDir: String, compressionCodec: Option[String]) +private[spark] object EventLoggingListener { + val LOG_PREFIX = "EVENT_LOG_" + val COMPRESSION_CODEC_PREFIX = "COMPRESSION_CODEC_" + val APPLICATION_COMPLETE = "APPLICATION_COMPLETE" + + def isEventLogFile(fileName: String): Boolean = { + fileName.contains(LOG_PREFIX) + } + + def isCompressionCodecFile(fileName: String): Boolean = { + fileName.contains(COMPRESSION_CODEC_PREFIX) + } + + def isApplicationCompleteFile(fileName: String): Boolean = { + fileName == APPLICATION_COMPLETE + } + + def parseCompressionCodec(fileName: String): String = { + if (isCompressionCodecFile(fileName)) { + fileName.replaceAll(COMPRESSION_CODEC_PREFIX, "") + } else "" + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala index db76178b65501..8c61e0742bc1a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -31,38 +31,62 @@ import org.apache.spark.io.CompressionCodec import org.apache.spark.util.{JsonProtocol, Utils} /** - * An EventBus that replays logged events from persisted storage + * An EventBus that replays logged events from persisted storage. + * + * This class expects files to be appropriately prefixed as specified in EventLoggingListener. + * There exists a one-to-one mapping between ReplayListenerBus and event logging applications. */ -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) +private[spark] class ReplayListenerBus(logDir: String) extends SparkListenerBus with Logging { + private val fileSystem = Utils.getHadoopFileSystem(new URI(logDir)) + private var applicationComplete = false + private var compressionCodec: Option[CompressionCodec] = None + private var logPaths = Array[Path]() + private var started = false + private var replayed = false /** - * Return a list of paths representing log files in the given directory. + * Prepare state for reading event logs. + * + * This gathers relevant files in the given directory and extracts meaning from each category. + * More specifically, this involves looking for event logs, the compression codec file + * (if event logs are compressed), and the application completion file (if the application + * has run to completion). */ - 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) + def start() { + val filePaths = getFilePaths(logDir, fileSystem) + logPaths = filePaths.filter { file => EventLoggingListener.isEventLogFile(file.getName) } + compressionCodec = + filePaths.find { file => + EventLoggingListener.isCompressionCodecFile(file.getName) + }.map { file => + val codec = EventLoggingListener.parseCompressionCodec(file.getName) + val conf = new SparkConf + conf.set("spark.io.compression.codec", codec) + CompressionCodec.createCodec(conf) + } + applicationComplete = + filePaths.exists { file => + EventLoggingListener.isApplicationCompleteFile(file.getName) + } + started = true + } + + /** Return whether the associated application signaled completion. */ + def isApplicationComplete: Boolean = { + assert(started, "ReplayListenerBus not started yet") + applicationComplete } /** - * Replay each event in the order maintained in the given logs. + * Replay each event in the order maintained in the given logs. This should only be called + * exactly once. Return whether event logs are actually found. */ - def replay(logDir: String): Boolean = { - val fileSystem = Utils.getHadoopFileSystem(new URI(logDir)) - val logPaths = getLogFilePaths(logDir, fileSystem) + def replay(): Boolean = { + assert(started, "ReplayListenerBus must be started before replaying logged events") + assert(!replayed, "ReplayListenerBus cannot replay events more than once") + if (logPaths.length == 0) { + logWarning("Log path provided contains no log files: %s".format(logDir)) return false } @@ -72,15 +96,11 @@ private[spark] class ReplayListenerBus(conf: SparkConf) extends SparkListenerBus var fileStream: Option[InputStream] = None var bufferedStream: Option[InputStream] = None var compressStream: Option[InputStream] = None - var currentLine = "" + var currentLine = "" try { - currentLine = "" fileStream = Some(fileSystem.open(path)) bufferedStream = Some(new FastBufferedInputStream(fileStream.get)) - compressStream = - if (compressed) { - Some(compressionCodec.compressedInputStream(bufferedStream.get)) - } else bufferedStream + compressStream = Some(wrapForCompression(bufferedStream.get)) // Parse each line as an event and post it to all attached listeners val lines = Source.fromInputStream(compressStream.get).getLines() @@ -98,7 +118,33 @@ private[spark] class ReplayListenerBus(conf: SparkConf) extends SparkListenerBus compressStream.foreach(_.close()) } } - fileSystem.close() + + replayed = true true } + + /** Stop the file system. */ + def stop() { + fileSystem.close() + } + + /** If a compression codec is specified, wrap the given stream in a compression stream. */ + private def wrapForCompression(stream: InputStream): InputStream = { + compressionCodec.map { codec => codec.compressedInputStream(stream) }.getOrElse(stream) + } + + /** Return a list of paths representing files found in the given directory. */ + private def getFilePaths(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("No files are found in the given log directory: %s".format(logDir)) + return Array[Path]() + } + logStatus.filter(!_.isDir).map(_.getPath).sortBy(_.getName) + } } 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 25b7472a99cdb..6a73ab10b5ba6 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 @@ -48,8 +48,9 @@ private[spark] class SparkDeploySchedulerBackend( val command = Command( "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs) val sparkHome = sc.getSparkHome() + val eventLogDir = sc.eventLogger.map { logger => Some(logger.logDir) }.getOrElse(None) val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, - sparkHome, sc.ui.appUIAddress, sc.eventLoggingInfo) + sparkHome, sc.ui.appUIAddress, eventLogDir) client = new AppClient(sc.env.actorSystem, masters, appDesc, this, conf) client.start() diff --git a/core/src/main/scala/org/apache/spark/storage/FileSegment.scala b/core/src/main/scala/org/apache/spark/storage/FileSegment.scala index 555486830a769..132502b75f8cd 100644 --- a/core/src/main/scala/org/apache/spark/storage/FileSegment.scala +++ b/core/src/main/scala/org/apache/spark/storage/FileSegment.scala @@ -23,6 +23,6 @@ import java.io.File * References a particular segment of a file (potentially the entire file), * based off an offset and a length. */ -private[spark] class FileSegment(val file: File, val offset: Long, val length : Long) { +private[spark] class FileSegment(val file: File, val offset: Long, val length: Long) { override def toString = "(name=%s, offset=%d, length=%d)".format(file.getName, offset, length) } 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 dc1b0c986c1d7..dc457a1ddcf06 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -39,8 +39,8 @@ private[spark] class SparkUI( extends WebUI("SparkUI") with 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) + def this(listenerBus: SparkListenerBus, appName: String, basePath: String) = + this(null, new SparkConf, listenerBus, appName, basePath) // If SparkContext is not provided, assume the associated application is not live val live = sc != null diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index f07962096a32c..479f9775a1ed8 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -46,7 +46,7 @@ class FileLogger( private val DATE_FORMAT = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") private val fileSystem = Utils.getHadoopFileSystem(new URI(logDir)) - private var fileIndex = 0 + var fileIndex = 0 // Only used if compression is enabled private lazy val compressionCodec = CompressionCodec.createCodec(conf) @@ -54,10 +54,9 @@ class FileLogger( // Only defined if the file system scheme is not local private var hadoopDataStream: Option[FSDataOutputStream] = None - private var writer: Option[PrintWriter] = { - createLogDir() - Some(createWriter()) - } + private var writer: Option[PrintWriter] = None + + createLogDir() /** * Create a logging directory with the given path. @@ -81,8 +80,8 @@ class FileLogger( /** * Create a new writer for the file identified by the given path. */ - private def createWriter(): PrintWriter = { - val logPath = logDir + "/" + fileIndex + private def createWriter(fileName: String): PrintWriter = { + val logPath = logDir + "/" + fileName val uri = new URI(logPath) /* The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844). @@ -144,13 +143,17 @@ class FileLogger( } /** - * Start a writer for a new file if one does not already exit. + * Start a writer for a new file, closing the existing one if it exists. + * @param fileName Name of the new file, defaulting to the file index if not provided. */ - def start() { - writer.getOrElse { - fileIndex += 1 - writer = Some(createWriter()) + def newFile(fileName: String = "") { + fileIndex += 1 + writer.foreach(_.close()) + val name = fileName match { + case "" => fileIndex.toString + case _ => fileName } + writer = Some(createWriter(name)) } /** From d5154da443058f3d1adfafd1a449faff03ca5056 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 21 Mar 2014 23:28:06 -0700 Subject: [PATCH 05/18] Styling and comments --- .../spark/deploy/history/HistoryServer.scala | 37 +++++++++++++------ .../history/HistoryServerArguments.scala | 15 +++----- .../spark/deploy/history/IndexPage.scala | 9 ++--- .../spark/scheduler/ReplayListenerBus.scala | 2 +- .../scala/org/apache/spark/ui/WebUI.scala | 2 +- 5 files changed, 36 insertions(+), 29 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index b36abbd9d4ff4..c789d9bad2b92 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -33,25 +33,28 @@ import org.apache.spark.util.Utils import org.apache.spark.scheduler.{ApplicationListener, ReplayListenerBus} /** - * A web server that re-renders SparkUIs of finished applications. + * A web server that renders SparkUIs of finished applications. * * For the standalone mode, MasterWebUI already achieves this functionality. Thus, the * main use case of the HistoryServer is in other deploy modes (e.g. Yarn or Mesos). * * The logging directory structure is as follows: Within the given base directory, each - * application's event logs are maintained in the application's own sub-directory. + * application's event logs are maintained in the application's own sub-directory. This + * is the same structure as maintained in the event log write code path in + * EventLoggingListener. * * @param baseLogDir The base directory in which event logs are found * @param requestedPort The requested port to which this server is to be bound */ -class HistoryServer(val baseLogDir: String, requestedPort: Int, conf: SparkConf) +class HistoryServer(val baseLogDir: String, requestedPort: Int) extends SparkUIContainer("History Server") with Logging { + private val fileSystem = Utils.getHadoopFileSystem(new URI(baseLogDir)) private val host = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(Utils.localHostName()) private val port = requestedPort - private val indexPage = new IndexPage(this) - private val fileSystem = Utils.getHadoopFileSystem(new URI(baseLogDir)) + private val conf = new SparkConf private val securityManager = new SecurityManager(conf) + private val indexPage = new IndexPage(this) // A timestamp of when the disk was last accessed to check for log updates private var lastLogCheck = -1L @@ -84,8 +87,8 @@ class HistoryServer(val baseLogDir: String, requestedPort: Int, conf: SparkConf) * from the application's event logs, attaches this UI to itself, and stores metadata * information for this application. * - * If the logs for an existing finished application are no longer found, remove all - * associated information and detach the SparkUI. + * If the logs for an existing finished application are no longer found, the server + * removes all associated information and detaches the SparkUI. */ def checkForLogs() { if (logCheckReady) { @@ -137,7 +140,7 @@ class HistoryServer(val baseLogDir: String, requestedPort: Int, conf: SparkConf) if (success) { attachUI(ui) val appName = if (appListener.applicationStarted) appListener.appName else appId - ui.setAppName("%s (history)".format(appName)) + ui.setAppName("%s (finished)".format(appName)) val startTime = appListener.startTime val endTime = appListener.endTime val info = ApplicationHistoryInfo(appName, startTime, endTime, lastUpdated, logPath, ui) @@ -155,7 +158,7 @@ class HistoryServer(val baseLogDir: String, requestedPort: Int, conf: SparkConf) /** Return the address of this server. */ def getAddress = "http://" + host + ":" + boundPort - /** Return when this directory is last modified. */ + /** Return when this directory was last modified. */ private def getModificationTime(dir: FileStatus): Long = { val logFiles = fileSystem.listStatus(dir.getPath) if (logFiles != null) { @@ -171,6 +174,16 @@ class HistoryServer(val baseLogDir: String, requestedPort: Int, conf: SparkConf) } } +/** + * The recommended way of starting and stopping a HistoryServer is through the scripts + * start-history-server.sh and stop-history-server.sh. The path to a base log directory + * is must be specified, while the requested UI port is optional. For example: + * + * ./sbin/spark-history-server.sh /tmp/spark-events 18080 + * ./sbin/spark-history-server.sh hdfs://1.2.3.4:9000/spark-events + * + * This launches the HistoryServer as a Spark daemon. + */ object HistoryServer { val STATIC_RESOURCE_DIR = SparkUI.STATIC_RESOURCE_DIR @@ -178,13 +191,13 @@ object HistoryServer { val UPDATE_INTERVAL_SECONDS = 5 def main(argStrings: Array[String]) { - val conf = new SparkConf - val args = new HistoryServerArguments(argStrings, conf) - val server = new HistoryServer(args.logDir, args.port, conf) + val args = new HistoryServerArguments(argStrings) + val server = new HistoryServer(args.logDir, args.port) server.bind() // Wait until the end of the world... or if the HistoryServer process is manually stopped while(true) { Thread.sleep(Int.MaxValue) } + server.stop() } } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala index c142b18b94aea..d221ad32bb048 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala @@ -19,20 +19,20 @@ package org.apache.spark.deploy.history import java.net.URI -import org.apache.spark.SparkConf -import org.apache.spark.util.{Utils, IntParam} import org.apache.hadoop.fs.Path +import org.apache.spark.util.{IntParam, Utils} + /** * Command-line parser for the master. */ -private[spark] class HistoryServerArguments(args: Array[String], conf: SparkConf) { +private[spark] class HistoryServerArguments(args: Array[String]) { var port = 18080 var logDir = "" parse(args.toList) - def parse(args: List[String]): Unit = { + private def parse(args: List[String]): Unit = { args match { case ("--port" | "-p") :: IntParam(value) :: tail => port = value @@ -53,7 +53,7 @@ private[spark] class HistoryServerArguments(args: Array[String], conf: SparkConf validateLogDir() } - def validateLogDir() { + private def validateLogDir() { if (logDir == "") { System.err.println("Logging directory must be specified.") printUsageAndExit(1) @@ -66,10 +66,7 @@ private[spark] class HistoryServerArguments(args: Array[String], conf: SparkConf } } - /** - * Print usage and exit JVM with the given exit code. - */ - def printUsageAndExit(exitCode: Int) { + private def printUsageAndExit(exitCode: Int) { System.err.println( "Usage: HistoryServer [options]\n" + "\n" + diff --git a/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala index 19713c75679ab..6a251978cf643 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala @@ -17,8 +17,6 @@ package org.apache.spark.deploy.history -import java.text.SimpleDateFormat -import java.util.Date import javax.servlet.http.HttpServletRequest import scala.xml.Node @@ -26,7 +24,6 @@ import scala.xml.Node import org.apache.spark.ui.{UIUtils, WebUI} private[spark] class IndexPage(parent: HistoryServer) { - private val dateFmt = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") def render(request: HttpServletRequest): Seq[Node] = { parent.checkForLogs() @@ -59,12 +56,12 @@ private[spark] class IndexPage(parent: HistoryServer) { private def appRow(info: ApplicationHistoryInfo): Seq[Node] = { val appName = if (info.started) info.name else parent.getAppId(info.logPath) val uiAddress = parent.getAddress + info.ui.basePath - val startTime = if (info.started) dateFmt.format(new Date(info.startTime)) else "Not started" - val endTime = if (info.finished) dateFmt.format(new Date(info.endTime)) else "Not finished" + val startTime = if (info.started) WebUI.formatDate(info.startTime) else "Not started" + val endTime = if (info.finished) WebUI.formatDate(info.endTime) else "Not finished" val difference = if (info.started && info.finished) info.endTime - info.startTime else -1L val duration = if (difference > 0) WebUI.formatDuration(difference) else "---" val logDirectory = parent.getAppId(info.logPath) - val lastUpdated = dateFmt.format(new Date(info.lastUpdated)) + val lastUpdated = WebUI.formatDate(info.lastUpdated) {appName} {startTime} diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala index 8c61e0742bc1a..d28f6cc05aea4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -130,7 +130,7 @@ private[spark] class ReplayListenerBus(logDir: String) extends SparkListenerBus /** If a compression codec is specified, wrap the given stream in a compression stream. */ private def wrapForCompression(stream: InputStream): InputStream = { - compressionCodec.map { codec => codec.compressedInputStream(stream) }.getOrElse(stream) + compressionCodec.map(_.compressedInputStream(stream)).getOrElse(stream) } /** Return a list of paths representing files found in the given directory. */ diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala index 586245abdedb5..2cc7582eca8a3 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -64,6 +64,6 @@ private[spark] object WebUI { return "%.0f min".format(minutes) } val hours = minutes / 60 - return "%.1f h".format(hours) + "%.1f h".format(hours) } } From 1b2f391be6e2c56469ac5a2f00354ee41e11de11 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 24 Mar 2014 14:11:13 -0700 Subject: [PATCH 06/18] Minor changes Rename ApplicationListener to ApplicationEventListener, and stop ReplaySparkListener in Master after use. --- .../org/apache/spark/deploy/history/HistoryServer.scala | 4 ++-- .../scala/org/apache/spark/deploy/master/Master.scala | 1 + ...tionListener.scala => ApplicationEventListener.scala} | 2 +- .../org/apache/spark/scheduler/ReplayListenerBus.scala | 9 ++++----- .../scheduler/cluster/SparkDeploySchedulerBackend.scala | 3 +-- core/src/main/scala/org/apache/spark/ui/SparkUI.scala | 4 +++- 6 files changed, 12 insertions(+), 11 deletions(-) rename core/src/main/scala/org/apache/spark/scheduler/{ApplicationListener.scala => ApplicationEventListener.scala} (96%) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index c789d9bad2b92..b933cff026dd1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -30,7 +30,7 @@ import org.apache.spark.deploy.SparkUIContainer import org.apache.spark.ui.SparkUI import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.Utils -import org.apache.spark.scheduler.{ApplicationListener, ReplayListenerBus} +import org.apache.spark.scheduler.{ApplicationEventListener, ReplayListenerBus} /** * A web server that renders SparkUIs of finished applications. @@ -131,7 +131,7 @@ class HistoryServer(val baseLogDir: String, requestedPort: Int) // If the application completion file is found if (replayBus.isApplicationComplete) { val ui = new SparkUI(replayBus, appId, "/history/%s".format(appId)) - val appListener = new ApplicationListener + val appListener = new ApplicationEventListener replayBus.addListener(appListener) // Do not call ui.bind() to avoid creating a new server for each application diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 0b9f165612526..f22f0452108a2 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 @@ -666,6 +666,7 @@ private[spark] class Master( // Do not call ui.bind() to avoid creating a new server for each application ui.start() val success = replayBus.replay() + replayBus.stop() if (success) Some(ui) else None } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ApplicationListener.scala b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala similarity index 96% rename from core/src/main/scala/org/apache/spark/scheduler/ApplicationListener.scala rename to core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala index 9a20ad1bb5ef4..7670647d15672 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ApplicationListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala @@ -24,7 +24,7 @@ package org.apache.spark.scheduler * SparkListenerApplicationEnd will be received. Otherwise, only the latest event * of each type will take effect. */ -private[spark] class ApplicationListener extends SparkListener { +private[spark] class ApplicationEventListener extends SparkListener { var appName = "" var startTime = -1L var endTime = -1L diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala index d28f6cc05aea4..c39482f6fbec1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -31,7 +31,7 @@ import org.apache.spark.io.CompressionCodec import org.apache.spark.util.{JsonProtocol, Utils} /** - * An EventBus that replays logged events from persisted storage. + * A SparkListenerBus that replays logged events from persisted storage. * * This class expects files to be appropriately prefixed as specified in EventLoggingListener. * There exists a one-to-one mapping between ReplayListenerBus and event logging applications. @@ -64,10 +64,9 @@ private[spark] class ReplayListenerBus(logDir: String) extends SparkListenerBus conf.set("spark.io.compression.codec", codec) CompressionCodec.createCodec(conf) } - applicationComplete = - filePaths.exists { file => - EventLoggingListener.isApplicationCompleteFile(file.getName) - } + applicationComplete = filePaths.exists { file => + EventLoggingListener.isApplicationCompleteFile(file.getName) + } started = true } 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 6a73ab10b5ba6..936e9db80573d 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 @@ -48,9 +48,8 @@ private[spark] class SparkDeploySchedulerBackend( val command = Command( "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs) val sparkHome = sc.getSparkHome() - val eventLogDir = sc.eventLogger.map { logger => Some(logger.logDir) }.getOrElse(None) val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, - sparkHome, sc.ui.appUIAddress, eventLogDir) + sparkHome, sc.ui.appUIAddress, sc.eventLogger.map(_.logDir)) client = new AppClient(sc.env.actorSystem, masters, appDesc, this, conf) client.start() 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 dc457a1ddcf06..5d2a5a69f9971 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -75,7 +75,9 @@ private[spark] class SparkUI( // Maintain executor storage status through Spark events val storageStatusListener = new StorageStatusListener - def setAppName(name: String) = appName = name + def setAppName(name: String) { + appName = name + } /** Initialize all components of the server */ def start() { From 067074395f8f9fbabf101f4db938781b8e99ccea Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 31 Mar 2014 15:51:19 -0700 Subject: [PATCH 07/18] Decouple page rendering from loading files from disk ... by making disk accesses asynchronous. This is a performance and UX win in case we're loading a huge file. --- .../spark/deploy/history/HistoryServer.scala | 44 +++++++++++-------- 1 file changed, 26 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index b933cff026dd1..22f26b59cd95b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -21,6 +21,9 @@ import java.net.URI import javax.servlet.http.HttpServletRequest import scala.collection.mutable +import scala.concurrent._ +import scala.concurrent.ExecutionContext.Implicits.global +import scala.util.{Failure, Success} import org.apache.hadoop.fs.{FileStatus, Path} import org.eclipse.jetty.servlet.ServletContextHandler @@ -81,7 +84,7 @@ class HistoryServer(val baseLogDir: String, requestedPort: Int) } /** - * Check for any updates to event logs in the base directory. + * Asynchronously check for any updates to event logs in the base directory. * * If a new finished application is found, the server renders the associated SparkUI * from the application's event logs, attaches this UI to itself, and stores metadata @@ -93,27 +96,32 @@ class HistoryServer(val baseLogDir: String, requestedPort: Int) def checkForLogs() { if (logCheckReady) { lastLogCheck = System.currentTimeMillis - val logStatus = fileSystem.listStatus(new Path(baseLogDir)) - val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]() - - // Render SparkUI for any new completed applications - logDirs.foreach { dir => - val path = dir.getPath.toString - val appId = getAppId(path) - val lastUpdated = getModificationTime(dir) - if (!appIdToInfo.contains(appId)) { - maybeRenderUI(appId, path, lastUpdated) + val asyncCheck = future { + val logStatus = fileSystem.listStatus(new Path(baseLogDir)) + val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]() + + // Render SparkUI for any new completed applications + logDirs.foreach { dir => + val path = dir.getPath.toString + val appId = getAppId(path) + val lastUpdated = getModificationTime(dir) + if (!appIdToInfo.contains(appId)) { + maybeRenderUI(appId, path, lastUpdated) + } } - } - // Remove any outdated SparkUIs - val appIds = logDirs.map { dir => getAppId(dir.getPath.toString) } - appIdToInfo.foreach { case (appId, info) => - if (!appIds.contains(appId)) { - detachUI(info.ui) - appIdToInfo.remove(appId) + // Remove any outdated SparkUIs + val appIds = logDirs.map { dir => getAppId(dir.getPath.toString) } + appIdToInfo.foreach { case (appId, info) => + if (!appIds.contains(appId)) { + detachUI(info.ui) + appIdToInfo.remove(appId) + } } } + asyncCheck.onFailure { case t => + logError("Unable to synchronize HistoryServer with files on disk: ", t) + } } } From 81b568ba753175c28636d674182b030fb41ce1f2 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 31 Mar 2014 16:31:41 -0700 Subject: [PATCH 08/18] Fix strange error messages... --- .../scala/org/apache/spark/deploy/history/HistoryServer.scala | 2 +- .../scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala | 2 +- .../scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala | 2 +- core/src/main/scala/org/apache/spark/ui/SparkUI.scala | 4 ++-- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 22f26b59cd95b..48ca6cfe05939 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -77,7 +77,7 @@ class HistoryServer(val baseLogDir: String, requestedPort: Int) logInfo("Started HistoryServer at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => - logError("Failed to create HistoryServer", e) + logError("Failed to bind HistoryServer", e) System.exit(1) } checkForLogs() 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 946eef782936b..ad76cfff955d1 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 @@ -66,7 +66,7 @@ class MasterWebUI(val master: Master, requestedPort: Int) logInfo("Started Master web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => - logError("Failed to create Master JettyUtils", e) + logError("Failed to create Master web UI", e) System.exit(1) } } 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 335a891493a46..0e2b0a4f86124 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 @@ -64,7 +64,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I logInfo("Started Worker web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => - logError("Failed to create Worker JettyUtils", e) + logError("Failed to create Worker web UI", e) System.exit(1) } } 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 5d2a5a69f9971..13c743989d43a 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -98,10 +98,10 @@ private[spark] class SparkUI( override def bind() { try { serverInfo = Some(startJettyServer(host, port, handlers, sc.conf)) - logInfo("Started Spark Web UI at http://%s:%d".format(host, boundPort)) + logInfo("Started Spark web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => - logError("Failed to create Spark JettyUtils", e) + logError("Failed to create Spark web UI", e) System.exit(1) } } From a3598deaf02ca92bd523ee7725d2f9639956a756 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Sun, 6 Apr 2014 21:00:36 -0700 Subject: [PATCH 09/18] Do not close file system with ReplayBus + fix bind address --- .../spark/deploy/history/HistoryServer.scala | 22 +++++++++++-------- .../apache/spark/deploy/master/Master.scala | 8 +++++-- .../spark/scheduler/ReplayListenerBus.scala | 10 ++++++--- .../scala/org/apache/spark/util/Utils.scala | 8 ++++++- 4 files changed, 33 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 48ca6cfe05939..fe46bc43cec01 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -17,13 +17,11 @@ package org.apache.spark.deploy.history -import java.net.URI import javax.servlet.http.HttpServletRequest import scala.collection.mutable import scala.concurrent._ import scala.concurrent.ExecutionContext.Implicits.global -import scala.util.{Failure, Success} import org.apache.hadoop.fs.{FileStatus, Path} import org.eclipse.jetty.servlet.ServletContextHandler @@ -52,8 +50,9 @@ import org.apache.spark.scheduler.{ApplicationEventListener, ReplayListenerBus} class HistoryServer(val baseLogDir: String, requestedPort: Int) extends SparkUIContainer("History Server") with Logging { - private val fileSystem = Utils.getHadoopFileSystem(new URI(baseLogDir)) - private val host = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(Utils.localHostName()) + private val fileSystem = Utils.getHadoopFileSystem(baseLogDir) + private val bindHost = Utils.localHostName() + private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(bindHost) private val port = requestedPort private val conf = new SparkConf private val securityManager = new SecurityManager(conf) @@ -73,8 +72,8 @@ class HistoryServer(val baseLogDir: String, requestedPort: Int) /** Bind to the HTTP server behind this web interface */ override def bind() { try { - serverInfo = Some(startJettyServer(host, port, handlers, conf)) - logInfo("Started HistoryServer at http://%s:%d".format(host, boundPort)) + serverInfo = Some(startJettyServer(bindHost, port, handlers, conf)) + logInfo("Started HistoryServer at http://%s:%d".format(publicHost, boundPort)) } catch { case e: Exception => logError("Failed to bind HistoryServer", e) @@ -133,7 +132,7 @@ class HistoryServer(val baseLogDir: String, requestedPort: Int) * which case the server proceeds to render the SparkUI. Otherwise, the server does nothing. */ private def maybeRenderUI(appId: String, logPath: String, lastUpdated: Long) { - val replayBus = new ReplayListenerBus(logPath) + val replayBus = new ReplayListenerBus(logPath, fileSystem) replayBus.start() // If the application completion file is found @@ -157,14 +156,19 @@ class HistoryServer(val baseLogDir: String, requestedPort: Int) } else { logWarning("Skipping incomplete application: %s".format(logPath)) } - replayBus.stop() + } + + /** Stop the server and close the file system. */ + override def stop() { + super.stop() + fileSystem.close() } /** Parse app ID from the given log path. */ def getAppId(logPath: String): String = logPath.split("/").last /** Return the address of this server. */ - def getAddress = "http://" + host + ":" + boundPort + def getAddress = "http://" + publicHost + ":" + boundPort /** Return when this directory was last modified. */ private def getModificationTime(dir: FileStatus): Long = { 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 f22f0452108a2..f1cd2857fd1b9 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 @@ -29,6 +29,7 @@ import akka.actor._ import akka.pattern.ask import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} import akka.serialization.SerializationExtension +import org.apache.hadoop.fs.FileSystem import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.{ApplicationDescription, DriverDescription, ExecutorState} @@ -45,7 +46,8 @@ private[spark] class Master( host: String, port: Int, webUiPort: Int, - val securityMgr: SecurityManager) extends Actor with Logging { + val securityMgr: SecurityManager) + extends Actor with Logging { import context.dispatcher // to use Akka's scheduler.schedule() @@ -71,6 +73,7 @@ private[spark] class Master( var nextAppNumber = 0 val appIdToUI = new HashMap[String, SparkUI] + val fileSystems = new HashSet[FileSystem] val drivers = new HashSet[DriverInfo] val completedDrivers = new ArrayBuffer[DriverInfo] @@ -149,6 +152,7 @@ private[spark] class Master( override def postStop() { webUi.stop() + fileSystems.foreach(_.close()) masterMetricsSystem.stop() applicationMetricsSystem.stop() persistenceEngine.close() @@ -662,11 +666,11 @@ private[spark] class Master( val eventLogDir = app.desc.eventLogDir.getOrElse { return None } val replayBus = new ReplayListenerBus(eventLogDir) val ui = new SparkUI(replayBus, "%s (finished)".format(appName), "/history/%s".format(app.id)) + fileSystems += replayBus.fileSystem // Do not call ui.bind() to avoid creating a new server for each application ui.start() val success = replayBus.replay() - replayBus.stop() if (success) Some(ui) else None } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala index c39482f6fbec1..06ff32d2a0ef4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -18,7 +18,6 @@ package org.apache.spark.scheduler import java.io.InputStream -import java.net.URI import scala.io.Source @@ -36,8 +35,13 @@ import org.apache.spark.util.{JsonProtocol, Utils} * This class expects files to be appropriately prefixed as specified in EventLoggingListener. * There exists a one-to-one mapping between ReplayListenerBus and event logging applications. */ -private[spark] class ReplayListenerBus(logDir: String) extends SparkListenerBus with Logging { - private val fileSystem = Utils.getHadoopFileSystem(new URI(logDir)) +private[spark] class ReplayListenerBus( + logDir: String, + val fileSystem: FileSystem) + extends SparkListenerBus with Logging { + + def this(logDir: String) = this(logDir, Utils.getHadoopFileSystem(logDir)) + private var applicationComplete = false private var compressionCodec: Option[CompressionCodec] = None private var logPaths = Array[Path]() 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 4435b21a7505e..105dd222a4e48 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -26,7 +26,6 @@ import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadPoolExecutor} import scala.collection.JavaConversions._ import scala.collection.Map import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.SortedSet import scala.io.Source import scala.reflect.ClassTag @@ -1022,4 +1021,11 @@ private[spark] object Utils extends Logging { def getHadoopFileSystem(path: URI): FileSystem = { FileSystem.get(path, SparkHadoopUtil.get.newConfiguration()) } + + /** + * Return a Hadoop FileSystem with the scheme encoded in the given path. + */ + def getHadoopFileSystem(path: String): FileSystem = { + getHadoopFileSystem(new URI(path)) + } } From 248cb3d75d25f7ae5828393d87f2c38e89a81b52 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Sun, 6 Apr 2014 23:21:41 -0700 Subject: [PATCH 10/18] Limit number of live applications + add configurability --- bin/spark-class | 8 +- bin/spark-class2.cmd | 7 +- .../spark/deploy/history/HistoryServer.scala | 80 ++++++++++++++----- .../spark/deploy/history/IndexPage.scala | 6 +- .../apache/spark/deploy/master/Master.scala | 1 + 5 files changed, 76 insertions(+), 26 deletions(-) diff --git a/bin/spark-class b/bin/spark-class index 76fde3e448891..1b0d309cc5b1c 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -47,9 +47,9 @@ DEFAULT_MEM=${SPARK_MEM:-512m} SPARK_DAEMON_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS -Dspark.akka.logLifecycleEvents=true" -# Add java opts and memory settings for master, worker, executors, and repl. +# Add java opts and memory settings for master, worker, history server, executors, and repl. case "$1" in - # Master and Worker use SPARK_DAEMON_JAVA_OPTS (and specific opts) + SPARK_DAEMON_MEMORY. + # Master, Worker, and HistoryServer use SPARK_DAEMON_JAVA_OPTS (and specific opts) + SPARK_DAEMON_MEMORY. 'org.apache.spark.deploy.master.Master') OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS $SPARK_MASTER_OPTS" OUR_JAVA_MEM=${SPARK_DAEMON_MEMORY:-$DEFAULT_MEM} @@ -58,6 +58,10 @@ case "$1" in OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS $SPARK_WORKER_OPTS" OUR_JAVA_MEM=${SPARK_DAEMON_MEMORY:-$DEFAULT_MEM} ;; + 'org.apache.spark.deploy.history.HistoryServer') + OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS $SPARK_HISTORY_OPTS" + OUR_JAVA_MEM=${SPARK_DAEMON_MEMORY:-$DEFAULT_MEM} + ;; # Executors use SPARK_JAVA_OPTS + SPARK_EXECUTOR_MEMORY. 'org.apache.spark.executor.CoarseGrainedExecutorBackend') diff --git a/bin/spark-class2.cmd b/bin/spark-class2.cmd index f488cfdbeceb6..4302c1b6b7ff4 100755 --- a/bin/spark-class2.cmd +++ b/bin/spark-class2.cmd @@ -45,14 +45,17 @@ if "x%OUR_JAVA_MEM%"=="x" set OUR_JAVA_MEM=512m set SPARK_DAEMON_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% -Dspark.akka.logLifecycleEvents=true -rem Add java opts and memory settings for master, worker, executors, and repl. -rem Master and Worker use SPARK_DAEMON_JAVA_OPTS (and specific opts) + SPARK_DAEMON_MEMORY. +rem Add java opts and memory settings for master, worker, history server, executors, and repl. +rem Master, Worker and HistoryServer use SPARK_DAEMON_JAVA_OPTS (and specific opts) + SPARK_DAEMON_MEMORY. if "%1"=="org.apache.spark.deploy.master.Master" ( set OUR_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% %SPARK_MASTER_OPTS% if not "x%SPARK_DAEMON_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DAEMON_MEMORY% ) else if "%1"=="org.apache.spark.deploy.worker.Worker" ( set OUR_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% %SPARK_WORKER_OPTS% if not "x%SPARK_DAEMON_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DAEMON_MEMORY% +) else if "%1"=="org.apache.spark.deploy.history.HistoryServer" ( + set OUR_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% %SPARK_HISTORY_OPTS% + if not "x%SPARK_DAEMON_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DAEMON_MEMORY% rem Executors use SPARK_JAVA_OPTS + SPARK_EXECUTOR_MEMORY. ) else if "%1"=="org.apache.spark.executor.CoarseGrainedExecutorBackend" ( diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index fe46bc43cec01..897eab93e235f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -47,14 +47,18 @@ import org.apache.spark.scheduler.{ApplicationEventListener, ReplayListenerBus} * @param baseLogDir The base directory in which event logs are found * @param requestedPort The requested port to which this server is to be bound */ -class HistoryServer(val baseLogDir: String, requestedPort: Int) +class HistoryServer( + val baseLogDir: String, + requestedPort: Int, + conf: SparkConf) extends SparkUIContainer("History Server") with Logging { + import HistoryServer._ + private val fileSystem = Utils.getHadoopFileSystem(baseLogDir) private val bindHost = Utils.localHostName() private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(bindHost) private val port = requestedPort - private val conf = new SparkConf private val securityManager = new SecurityManager(conf) private val indexPage = new IndexPage(this) @@ -62,13 +66,17 @@ class HistoryServer(val baseLogDir: String, requestedPort: Int) private var lastLogCheck = -1L private val handlers = Seq[ServletContextHandler]( - createStaticHandler(HistoryServer.STATIC_RESOURCE_DIR, "/static"), + createStaticHandler(STATIC_RESOURCE_DIR, "/static"), createServletHandler("/", (request: HttpServletRequest) => indexPage.render(request), securityMgr = securityManager) ) + // A mapping of application ID to its history information, which includes the rendered UI val appIdToInfo = mutable.HashMap[String, ApplicationHistoryInfo]() + // A set of recently removed applications that the server should avoid re-rendering + val appIdBlacklist = mutable.HashSet[String]() + /** Bind to the HTTP server behind this web interface */ override def bind() { try { @@ -99,22 +107,28 @@ class HistoryServer(val baseLogDir: String, requestedPort: Int) val logStatus = fileSystem.listStatus(new Path(baseLogDir)) val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]() + // Forget about any SparkUIs that can no longer be found + val appIds = logDirs.map { dir => getAppId(dir.getPath.toString) } + appIdToInfo.foreach { case (appId, info) => + if (!appIds.contains(appId)) { + detachUI(info.ui) + appIdToInfo.remove(appId) + appIdBlacklist.clear() + } + } + appIdBlacklist.retain(appIds.contains) + // Render SparkUI for any new completed applications logDirs.foreach { dir => val path = dir.getPath.toString val appId = getAppId(path) val lastUpdated = getModificationTime(dir) - if (!appIdToInfo.contains(appId)) { + if (!appIdToInfo.contains(appId) && !appIdBlacklist.contains(appId)) { maybeRenderUI(appId, path, lastUpdated) } - } - - // Remove any outdated SparkUIs - val appIds = logDirs.map { dir => getAppId(dir.getPath.toString) } - appIdToInfo.foreach { case (appId, info) => - if (!appIds.contains(appId)) { - detachUI(info.ui) - appIdToInfo.remove(appId) + // If the cap is reached, remove the least recently updated application + if (appIdToInfo.size > RETAINED_APPLICATIONS) { + removeOldestApp() } } } @@ -144,14 +158,17 @@ class HistoryServer(val baseLogDir: String, requestedPort: Int) // Do not call ui.bind() to avoid creating a new server for each application ui.start() val success = replayBus.replay() - if (success) { + if (success && appListener.applicationStarted) { attachUI(ui) - val appName = if (appListener.applicationStarted) appListener.appName else appId + val appName = appListener.appName ui.setAppName("%s (finished)".format(appName)) val startTime = appListener.startTime val endTime = appListener.endTime val info = ApplicationHistoryInfo(appName, startTime, endTime, lastUpdated, logPath, ui) appIdToInfo(appId) = info + } else { + logWarning("Reconstructing application UI was unsuccessful. Either no event logs were" + + "found or the event signaling application start is missing: %s".format(logPath)) } } else { logWarning("Skipping incomplete application: %s".format(logPath)) @@ -168,7 +185,10 @@ class HistoryServer(val baseLogDir: String, requestedPort: Int) def getAppId(logPath: String): String = logPath.split("/").last /** Return the address of this server. */ - def getAddress = "http://" + publicHost + ":" + boundPort + def getAddress: String = "http://" + publicHost + ":" + boundPort + + /** Return the total number of application logs found, blacklisted or not. */ + def getTotalApplications: Int = appIdToInfo.size + appIdBlacklist.size /** Return when this directory was last modified. */ private def getModificationTime(dir: FileStatus): Long = { @@ -180,12 +200,26 @@ class HistoryServer(val baseLogDir: String, requestedPort: Int) } } + /** + * Remove the oldest application and detach its associated UI. As an optimization, add the + * application to a blacklist to avoid re-rendering it the next time. + */ + private def removeOldestApp() { + val appToRemove = appIdToInfo.toSeq.minBy { case (_, info) => info.lastUpdated } + appToRemove match { case (id, info) => + appIdToInfo.remove(id) + detachUI(info.ui) + appIdBlacklist.add(id) + } + } + /** Return whether the last log check has happened sufficiently long ago. */ private def logCheckReady: Boolean = { - System.currentTimeMillis - lastLogCheck > HistoryServer.UPDATE_INTERVAL_SECONDS * 1000 + System.currentTimeMillis - lastLogCheck > UPDATE_INTERVAL_SECONDS * 1000 } } + /** * The recommended way of starting and stopping a HistoryServer is through the scripts * start-history-server.sh and stop-history-server.sh. The path to a base log directory @@ -197,14 +231,19 @@ class HistoryServer(val baseLogDir: String, requestedPort: Int) * This launches the HistoryServer as a Spark daemon. */ object HistoryServer { - val STATIC_RESOURCE_DIR = SparkUI.STATIC_RESOURCE_DIR + private val conf = new SparkConf - // Minimum interval between each check for logs, which requires a disk access - val UPDATE_INTERVAL_SECONDS = 5 + // Minimum interval between each check for logs, which requires a disk access (seconds) + private val UPDATE_INTERVAL_SECONDS = conf.getInt("spark.history.updateInterval", 5) + + // How many applications to retain + private val RETAINED_APPLICATIONS = conf.getInt("spark.deploy.retainedApplications", 20) + + private val STATIC_RESOURCE_DIR = SparkUI.STATIC_RESOURCE_DIR def main(argStrings: Array[String]) { val args = new HistoryServerArguments(argStrings) - val server = new HistoryServer(args.logDir, args.port) + val server = new HistoryServer(args.logDir, args.port, conf) server.bind() // Wait until the end of the world... or if the HistoryServer process is manually stopped @@ -213,6 +252,7 @@ object HistoryServer { } } + private[spark] case class ApplicationHistoryInfo( name: String, startTime: Long, diff --git a/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala index 6a251978cf643..08ea4730d6a8d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala @@ -36,8 +36,10 @@ private[spark] class IndexPage(parent: HistoryServer) {
  • Event Log Location: {parent.baseLogDir}
  • -

    -

    Finished Applications

    {appTable} +

    + Showing {parent.appIdToInfo.size}/{parent.getTotalApplications} Finished Applications +

    + {appTable}
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 f1cd2857fd1b9..d940bc44569fd 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 @@ -670,6 +670,7 @@ private[spark] class Master( // Do not call ui.bind() to avoid creating a new server for each application ui.start() + replayBus.start() val success = replayBus.replay() if (success) Some(ui) else None } From 19e1fb42045af5b56569c7e6ba183ee577e9c2df Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 9 Apr 2014 00:57:13 -0700 Subject: [PATCH 11/18] Address Thomas' comments The biggest changes here include: (1) Periodically checking for event log updates in the background, instead of on refresh, (2) Use a Long instead of a linearly scaling HashSet to keep track of the applications whose UIs are purposefully not rendered, and (3) Adding Spark user as a new column. This includes adding a new field to the ApplicationStart event. --- .../scala/org/apache/spark/SparkContext.scala | 2 +- .../spark/deploy/history/HistoryServer.scala | 169 ++++++++++++------ .../spark/deploy/history/IndexPage.scala | 22 ++- .../scheduler/ApplicationEventListener.scala | 2 + .../spark/scheduler/ReplayListenerBus.scala | 26 +-- .../spark/scheduler/SparkListener.scala | 3 +- .../org/apache/spark/util/JsonProtocol.scala | 6 +- .../apache/spark/util/JsonProtocolSuite.scala | 6 +- 8 files changed, 156 insertions(+), 80 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index fd2aea86c31d5..1af404496a158 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1103,7 +1103,7 @@ class SparkContext( /** Post the application start event */ private def postApplicationStart() { - listenerBus.post(SparkListenerApplicationStart(appName, startTime)) + listenerBus.post(SparkListenerApplicationStart(appName, startTime, sparkUser)) } /** diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 897eab93e235f..c208eedc75bdb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -20,8 +20,6 @@ package org.apache.spark.deploy.history import javax.servlet.http.HttpServletRequest import scala.collection.mutable -import scala.concurrent._ -import scala.concurrent.ExecutionContext.Implicits.global import org.apache.hadoop.fs.{FileStatus, Path} import org.eclipse.jetty.servlet.ServletContextHandler @@ -56,14 +54,44 @@ class HistoryServer( import HistoryServer._ private val fileSystem = Utils.getHadoopFileSystem(baseLogDir) - private val bindHost = Utils.localHostName() - private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(bindHost) + private val localHost = Utils.localHostName() + private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHost) private val port = requestedPort private val securityManager = new SecurityManager(conf) private val indexPage = new IndexPage(this) // A timestamp of when the disk was last accessed to check for log updates - private var lastLogCheck = -1L + private var lastLogCheckTime = -1L + + // If an application is last updated after this threshold, then its UI is retained + private var updateTimeThreshold = -1L + + // Number of applications hidden from the UI because the application limit has been reached + private var numApplicationsHidden = 0 + + @volatile private var stopped = false + + /** + * A background thread that periodically checks for event log updates on disk. + * + * If a log check is invoked manually in the middle of a period, this thread re-adjusts the + * time at which it performs the next log check to maintain the same period as before. + */ + private val logCheckingThread = new Thread { + override def run() { + while (!stopped) { + val now = System.currentTimeMillis + if (now - lastLogCheckTime > UPDATE_INTERVAL_MS) { + checkForLogs() + Thread.sleep(UPDATE_INTERVAL_MS) + } else { + // If the user has manually checked for logs recently, wait until + // UPDATE_INTERVAL_MS after the last check time + Thread.sleep(lastLogCheckTime + UPDATE_INTERVAL_MS - now) + } + } + } + } private val handlers = Seq[ServletContextHandler]( createStaticHandler(STATIC_RESOURCE_DIR, "/static"), @@ -74,13 +102,20 @@ class HistoryServer( // A mapping of application ID to its history information, which includes the rendered UI val appIdToInfo = mutable.HashMap[String, ApplicationHistoryInfo]() - // A set of recently removed applications that the server should avoid re-rendering - val appIdBlacklist = mutable.HashSet[String]() + /** + * Start the history server. + * + * This starts a background thread that periodically synchronizes information displayed on + * this UI with the event logs in the provided base directory. + */ + def start() { + logCheckingThread.start() + } - /** Bind to the HTTP server behind this web interface */ + /** Bind to the HTTP server behind this web interface. */ override def bind() { try { - serverInfo = Some(startJettyServer(bindHost, port, handlers, conf)) + serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, conf)) logInfo("Started HistoryServer at http://%s:%d".format(publicHost, boundPort)) } catch { case e: Exception => @@ -91,7 +126,8 @@ class HistoryServer( } /** - * Asynchronously check for any updates to event logs in the base directory. + * Check for any updates to event logs in the base directory. This is only effective once + * the server has been bound. * * If a new finished application is found, the server renders the associated SparkUI * from the application's event logs, attaches this UI to itself, and stores metadata @@ -100,41 +136,54 @@ class HistoryServer( * If the logs for an existing finished application are no longer found, the server * removes all associated information and detaches the SparkUI. */ - def checkForLogs() { - if (logCheckReady) { - lastLogCheck = System.currentTimeMillis - val asyncCheck = future { + def checkForLogs() = synchronized { + if (serverInfo.isDefined) { + lastLogCheckTime = System.currentTimeMillis + logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTime)) + try { val logStatus = fileSystem.listStatus(new Path(baseLogDir)) val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]() // Forget about any SparkUIs that can no longer be found - val appIds = logDirs.map { dir => getAppId(dir.getPath.toString) } + val mostRecentAppIds = logDirs.map { dir => getAppId(dir.getPath.toString) } appIdToInfo.foreach { case (appId, info) => - if (!appIds.contains(appId)) { + if (!mostRecentAppIds.contains(appId)) { detachUI(info.ui) appIdToInfo.remove(appId) - appIdBlacklist.clear() + updateTimeThreshold = -1L } } - appIdBlacklist.retain(appIds.contains) + + // Keep track of the number of applications hidden from the UI this round + var _numApplicationsHidden = 0 // Render SparkUI for any new completed applications logDirs.foreach { dir => val path = dir.getPath.toString val appId = getAppId(path) val lastUpdated = getModificationTime(dir) - if (!appIdToInfo.contains(appId) && !appIdBlacklist.contains(appId)) { - maybeRenderUI(appId, path, lastUpdated) + if (!appIdToInfo.contains(appId)) { + if (lastUpdated > updateTimeThreshold) { + maybeRenderUI(appId, path, lastUpdated) + } else { + // This application was previously blacklisted due to the application limit + _numApplicationsHidden += 1 + } } // If the cap is reached, remove the least recently updated application if (appIdToInfo.size > RETAINED_APPLICATIONS) { removeOldestApp() + _numApplicationsHidden += 1 } } + + numApplicationsHidden = _numApplicationsHidden + + } catch { + case t: Throwable => logError("Exception in checking for event log updates", t) } - asyncCheck.onFailure { case t => - logError("Unable to synchronize HistoryServer with files on disk: ", t) - } + } else { + logWarning("Attempted to check for event log updates before binding the server.") } } @@ -161,11 +210,12 @@ class HistoryServer( if (success && appListener.applicationStarted) { attachUI(ui) val appName = appListener.appName - ui.setAppName("%s (finished)".format(appName)) + val sparkUser = appListener.sparkUser val startTime = appListener.startTime val endTime = appListener.endTime - val info = ApplicationHistoryInfo(appName, startTime, endTime, lastUpdated, logPath, ui) - appIdToInfo(appId) = info + ui.setAppName("%s (finished)".format(appName)) + appIdToInfo(appId) = + ApplicationHistoryInfo(appName, startTime, endTime, lastUpdated, sparkUser, logPath, ui) } else { logWarning("Reconstructing application UI was unsuccessful. Either no event logs were" + "found or the event signaling application start is missing: %s".format(logPath)) @@ -175,9 +225,27 @@ class HistoryServer( } } + /** + * Remove the oldest application and detach its associated UI. + * + * As an optimization, record the last updated time of this application as the minimum + * update time threshold. Only applications with a last updated time that exceeds this + * threshold will be retained by the server. This avoids re-rendering an old application + * that is recently removed. + */ + private def removeOldestApp() { + val appToRemove = appIdToInfo.toSeq.minBy { case (_, info) => info.lastUpdated } + appToRemove match { case (id, info) => + appIdToInfo.remove(id) + detachUI(info.ui) + updateTimeThreshold = info.lastUpdated + } + } + /** Stop the server and close the file system. */ override def stop() { super.stop() + stopped = true fileSystem.close() } @@ -187,39 +255,26 @@ class HistoryServer( /** Return the address of this server. */ def getAddress: String = "http://" + publicHost + ":" + boundPort - /** Return the total number of application logs found, blacklisted or not. */ - def getTotalApplications: Int = appIdToInfo.size + appIdBlacklist.size + /** Return the total number of application logs found, whether or not the UI is retained. */ + def getTotalApplications: Int = appIdToInfo.size + numApplicationsHidden /** Return when this directory was last modified. */ private def getModificationTime(dir: FileStatus): Long = { - val logFiles = fileSystem.listStatus(dir.getPath) - if (logFiles != null) { - logFiles.map(_.getModificationTime).max - } else { - dir.getModificationTime - } - } - - /** - * Remove the oldest application and detach its associated UI. As an optimization, add the - * application to a blacklist to avoid re-rendering it the next time. - */ - private def removeOldestApp() { - val appToRemove = appIdToInfo.toSeq.minBy { case (_, info) => info.lastUpdated } - appToRemove match { case (id, info) => - appIdToInfo.remove(id) - detachUI(info.ui) - appIdBlacklist.add(id) + try { + val logFiles = fileSystem.listStatus(dir.getPath) + if (logFiles != null) { + logFiles.map(_.getModificationTime).max + } else { + dir.getModificationTime + } + } catch { + case t: Throwable => + logError("Exception in accessing modification time of %s".format(dir.getPath), t) + -1L } } - - /** Return whether the last log check has happened sufficiently long ago. */ - private def logCheckReady: Boolean = { - System.currentTimeMillis - lastLogCheck > UPDATE_INTERVAL_SECONDS * 1000 - } } - /** * The recommended way of starting and stopping a HistoryServer is through the scripts * start-history-server.sh and stop-history-server.sh. The path to a base log directory @@ -233,18 +288,19 @@ class HistoryServer( object HistoryServer { private val conf = new SparkConf - // Minimum interval between each check for logs, which requires a disk access (seconds) - private val UPDATE_INTERVAL_SECONDS = conf.getInt("spark.history.updateInterval", 5) + // Interval between each check for event log updates + val UPDATE_INTERVAL_MS = conf.getInt("spark.history.updateInterval", 10) * 1000 // How many applications to retain - private val RETAINED_APPLICATIONS = conf.getInt("spark.deploy.retainedApplications", 20) + val RETAINED_APPLICATIONS = conf.getInt("spark.deploy.retainedApplications", 250) - private val STATIC_RESOURCE_DIR = SparkUI.STATIC_RESOURCE_DIR + val STATIC_RESOURCE_DIR = SparkUI.STATIC_RESOURCE_DIR def main(argStrings: Array[String]) { val args = new HistoryServerArguments(argStrings) val server = new HistoryServer(args.logDir, args.port, conf) server.bind() + server.start() // Wait until the end of the world... or if the HistoryServer process is manually stopped while(true) { Thread.sleep(Int.MaxValue) } @@ -258,6 +314,7 @@ private[spark] case class ApplicationHistoryInfo( startTime: Long, endTime: Long, lastUpdated: Long, + sparkUser: String, logPath: String, ui: SparkUI) { def started = startTime != -1 diff --git a/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala index 08ea4730d6a8d..e66463eef08c6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala @@ -26,9 +26,6 @@ import org.apache.spark.ui.{UIUtils, WebUI} private[spark] class IndexPage(parent: HistoryServer) { def render(request: HttpServletRequest): Seq[Node] = { - parent.checkForLogs() - - // Populate app table, with most recently modified app first val appRows = parent.appIdToInfo.values.toSeq.sortBy { app => -app.lastUpdated } val appTable = UIUtils.listingTable(appHeader, appRow, appRows) val content = @@ -36,14 +33,20 @@ private[spark] class IndexPage(parent: HistoryServer) {
  • Event Log Location: {parent.baseLogDir}
  • -

    - Showing {parent.appIdToInfo.size}/{parent.getTotalApplications} Finished Applications -

    - {appTable}
+ { + if (parent.appIdToInfo.size > 0) { +

+ Showing {parent.appIdToInfo.size}/{parent.getTotalApplications} + Finished Application{if (parent.getTotalApplications > 1) "s" else ""} +

++ + appTable + } else { +

No Finished Applications Found

+ } + }
- UIUtils.basicSparkPage(content, "History Server") } @@ -52,6 +55,7 @@ private[spark] class IndexPage(parent: HistoryServer) { "Started", "Finished", "Duration", + "Spark User", "Log Directory", "Last Updated") @@ -62,6 +66,7 @@ private[spark] class IndexPage(parent: HistoryServer) { val endTime = if (info.finished) WebUI.formatDate(info.endTime) else "Not finished" val difference = if (info.started && info.finished) info.endTime - info.startTime else -1L val duration = if (difference > 0) WebUI.formatDuration(difference) else "---" + val sparkUser = if (info.started) info.sparkUser else "Unknown user" val logDirectory = parent.getAppId(info.logPath) val lastUpdated = WebUI.formatDate(info.lastUpdated) @@ -69,6 +74,7 @@ private[spark] class IndexPage(parent: HistoryServer) { {startTime} {endTime} {duration} + {sparkUser} {logDirectory} {lastUpdated} diff --git a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala index 7670647d15672..9f630e20e0b09 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala @@ -26,6 +26,7 @@ package org.apache.spark.scheduler */ private[spark] class ApplicationEventListener extends SparkListener { var appName = "" + var sparkUser = "" var startTime = -1L var endTime = -1L @@ -41,6 +42,7 @@ private[spark] class ApplicationEventListener extends SparkListener { override def onApplicationStart(applicationStart: SparkListenerApplicationStart) { appName = applicationStart.appName startTime = applicationStart.time + sparkUser = applicationStart.sparkUser } override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala index 06ff32d2a0ef4..0e403f06e0e95 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -138,16 +138,22 @@ private[spark] class ReplayListenerBus( /** Return a list of paths representing files found in the given directory. */ private def getFilePaths(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("No files are found in the given log directory: %s".format(logDir)) - return Array[Path]() + try { + 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("No files are found in the given log directory: %s".format(logDir)) + return Array[Path]() + } + logStatus.filter(!_.isDir).map(_.getPath).sortBy(_.getName) + } catch { + case t: Throwable => + logError("Exception in accessing log files in %s".format(logDir), t) + Array[Path]() } - logStatus.filter(!_.isDir).map(_.getPath).sortBy(_.getName) } } 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 5ca06abd1b5fc..5a0af00f4d0fd 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -62,7 +62,8 @@ case class SparkListenerBlockManagerRemoved(blockManagerId: BlockManagerId) case class SparkListenerUnpersistRDD(rddId: Int) extends SparkListenerEvent -case class SparkListenerApplicationStart(appName: String, time: Long) extends SparkListenerEvent +case class SparkListenerApplicationStart(appName: String, time: Long, sparkUser: String) + extends SparkListenerEvent case class SparkListenerApplicationEnd(time: Long) extends SparkListenerEvent diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 03884951ca386..e1feca45dbb90 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -164,7 +164,8 @@ private[spark] object JsonProtocol { def applicationStartToJson(applicationStart: SparkListenerApplicationStart): JValue = { ("Event" -> Utils.getFormattedClassName(applicationStart)) ~ ("App Name" -> applicationStart.appName) ~ - ("Timestamp" -> applicationStart.time) + ("Timestamp" -> applicationStart.time) ~ + ("User" -> applicationStart.sparkUser) } def applicationEndToJson(applicationEnd: SparkListenerApplicationEnd): JValue = { @@ -452,7 +453,8 @@ private[spark] object JsonProtocol { def applicationStartFromJson(json: JValue): SparkListenerApplicationStart = { val appName = (json \ "App Name").extract[String] val time = (json \ "Timestamp").extract[Long] - SparkListenerApplicationStart(appName, time) + val sparkUser = (json \ "User").extract[String] + SparkListenerApplicationStart(appName, time, sparkUser) } def applicationEndFromJson(json: JValue): SparkListenerApplicationEnd = { diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 029773b910009..86ec0eed930a7 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -52,7 +52,7 @@ class JsonProtocolSuite extends FunSuite { val blockManagerRemoved = SparkListenerBlockManagerRemoved( BlockManagerId("Scarce", "to be counted...", 100, 200)) val unpersistRdd = SparkListenerUnpersistRDD(12345) - val applicationStart = SparkListenerApplicationStart("The winner of all", 42L) + val applicationStart = SparkListenerApplicationStart("The winner of all", 42L, "Garfield") val applicationEnd = SparkListenerApplicationEnd(42L) testEvent(stageSubmitted, stageSubmittedJsonString) @@ -217,6 +217,7 @@ class JsonProtocolSuite extends FunSuite { case (e1: SparkListenerApplicationStart, e2: SparkListenerApplicationStart) => assert(e1.appName == e2.appName) assert(e1.time == e2.time) + assert(e1.sparkUser == e2.sparkUser) case (e1: SparkListenerApplicationEnd, e2: SparkListenerApplicationEnd) => assert(e1.time == e2.time) case (SparkListenerShutdown, SparkListenerShutdown) => @@ -566,7 +567,8 @@ class JsonProtocolSuite extends FunSuite { private val applicationStartJsonString = """ - {"Event":"SparkListenerApplicationStart","App Name":"The winner of all","Timestamp":42} + {"Event":"SparkListenerApplicationStart","App Name":"The winner of all","Timestamp":42, + "User":"Garfield"} """ private val applicationEndJsonString = From 2282300ddbc6e976f9c1126c9341685669d54f9e Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 9 Apr 2014 12:51:02 -0700 Subject: [PATCH 12/18] Add documentation for the HistoryServer --- .../spark/deploy/history/HistoryServer.scala | 4 +- docs/monitoring.md | 64 +++++++++++++++++-- 2 files changed, 62 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index c208eedc75bdb..b96865c95c52b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -76,6 +76,8 @@ class HistoryServer( * * If a log check is invoked manually in the middle of a period, this thread re-adjusts the * time at which it performs the next log check to maintain the same period as before. + * + * TODO: Add a mechanism to update manually. */ private val logCheckingThread = new Thread { override def run() { @@ -292,7 +294,7 @@ object HistoryServer { val UPDATE_INTERVAL_MS = conf.getInt("spark.history.updateInterval", 10) * 1000 // How many applications to retain - val RETAINED_APPLICATIONS = conf.getInt("spark.deploy.retainedApplications", 250) + val RETAINED_APPLICATIONS = conf.getInt("spark.history.retainedApplications", 250) val STATIC_RESOURCE_DIR = SparkUI.STATIC_RESOURCE_DIR diff --git a/docs/monitoring.md b/docs/monitoring.md index 15bfb041780da..ee7854ccd66fd 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -12,17 +12,71 @@ displays useful information about the application. This includes: * A list of scheduler stages and tasks * A summary of RDD sizes and memory usage -* Information about the running executors * Environmental information. +* Information about the running executors You can access this interface by simply opening `http://:4040` in a web browser. -If multiple SparkContexts are running on the same host, they will bind to succesive ports +If multiple SparkContexts are running on the same host, they will bind to successive ports beginning with 4040 (4041, 4042, etc). -Spark's Standalone Mode cluster manager also has its own -[web UI](spark-standalone.html#monitoring-and-logging). +Note that this information is only available for the duration of the application by default. +To view the web UI after the fact, set `spark.eventLog.enabled` to true before starting the +application. This configures Spark to log Spark events that encode the information displayed +in the UI to persisted storage. -Note that in both of these UIs, the tables are sortable by clicking their headers, +## Viewing After the Fact + +Spark's Standalone Mode cluster manager also has its own +[web UI](spark-standalone.html#monitoring-and-logging). If an application has logged events over +the course of its lifetime, then the Standalone master's web UI will automatically re-render the +application's UI after the application has finished. + +If Spark is run on Mesos or YARN, it is still possible to reconstruct the UI of a finished +application through Spark's history server, provided that the application's event logs exist. +You can start a the history server by executing: + + ./sbin/start-history-server.sh + +The base logging directory must be supplied, and should contain sub-directories that each +represents an application's event logs. This creates a web interface at +`http://:18080` by default, but the port can be changed by supplying an extra +parameter to the start script. The history server depends on the following variables: + + + + + + + + + + + +
Environment VariableMeaning
SPARK_DAEMON_MEMORYMemory to allocate to the history server. (default: 512m).
SPARK_DAEMON_JAVA_OPTSJVM options for the history server (default: none).
+ +Further, the history server can be configured as follows: + + + + + + + + + + + + + +
Property NameDefaultMeaning
spark.history.updateInterval10 + The period at which information displayed by this history server is updated. Each update + checks for any changes made to the event logs in persisted storage. +
spark.history.retainedApplications250 + The number of application UIs to retain. If this cap is exceeded, then the least recently + updated applications will be removed. +
+ +Note that in all of these UIs, the tables are sortable by clicking their headers, making it easy to identify slow tasks, data skew, etc. # Metrics From d02dbaa8818de2a4acd1d5621afb8382fb314a2c Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 9 Apr 2014 13:17:42 -0700 Subject: [PATCH 13/18] Expose Spark version and include it in event logs This allows us to deal with inconsistencies in event log version incompatibilities in the future. --- .../scala/org/apache/spark/SparkContext.scala | 5 +++ .../scheduler/EventLoggingListener.scala | 18 +++++++++-- .../spark/scheduler/ReplayListenerBus.scala | 31 ++++++++++++------- 3 files changed, 40 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 1ae811088c7ae..c25cee74bc91f 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -775,6 +775,9 @@ class SparkContext(config: SparkConf) extends Logging { listenerBus.addListener(listener) } + /** The version of Spark on which this application is running. */ + def version = SparkContext.SPARK_VERSION + /** * Return a map from the slave to the max memory available for caching and the remaining * memory available for caching. @@ -1213,6 +1216,8 @@ class SparkContext(config: SparkConf) extends Logging { */ object SparkContext extends Logging { + private[spark] val SPARK_VERSION = "1.0.0" + private[spark] val SPARK_JOB_DESCRIPTION = "spark.job.description" private[spark] val SPARK_JOB_GROUP_ID = "spark.jobGroup.id" diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index ead6a2904228b..c3173e2ab0859 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -19,7 +19,7 @@ package org.apache.spark.scheduler import org.json4s.jackson.JsonMethods._ -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.{Logging, SparkConf, SparkContext} import org.apache.spark.io.CompressionCodec import org.apache.spark.util.{JsonProtocol, FileLogger} @@ -58,6 +58,7 @@ private[spark] class EventLoggingListener(appName: String, conf: SparkConf) val codec = conf.get("spark.io.compression.codec", CompressionCodec.DEFAULT_COMPRESSION_CODEC) logger.newFile(COMPRESSION_CODEC_PREFIX + codec) } + logger.newFile(SPARK_VERSION_PREFIX + SparkContext.SPARK_VERSION) logger.newFile(LOG_PREFIX + logger.fileIndex) } @@ -111,22 +112,33 @@ private[spark] class EventLoggingListener(appName: String, conf: SparkConf) } private[spark] object EventLoggingListener { + val SPARK_VERSION_PREFIX = "SPARK_VERSION_" val LOG_PREFIX = "EVENT_LOG_" val COMPRESSION_CODEC_PREFIX = "COMPRESSION_CODEC_" val APPLICATION_COMPLETE = "APPLICATION_COMPLETE" + def isSparkVersionFile(fileName: String): Boolean = { + fileName.startsWith(SPARK_VERSION_PREFIX) + } + def isEventLogFile(fileName: String): Boolean = { - fileName.contains(LOG_PREFIX) + fileName.startsWith(LOG_PREFIX) } def isCompressionCodecFile(fileName: String): Boolean = { - fileName.contains(COMPRESSION_CODEC_PREFIX) + fileName.startsWith(COMPRESSION_CODEC_PREFIX) } def isApplicationCompleteFile(fileName: String): Boolean = { fileName == APPLICATION_COMPLETE } + def parseSparkVersion(fileName: String): String = { + if (isSparkVersionFile(fileName)) { + fileName.replaceAll(SPARK_VERSION_PREFIX, "") + } else "" + } + def parseCompressionCodec(fileName: String): String = { if (isCompressionCodecFile(fileName)) { fileName.replaceAll(COMPRESSION_CODEC_PREFIX, "") diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala index 0e403f06e0e95..617e956a56c54 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -43,6 +43,7 @@ private[spark] class ReplayListenerBus( def this(logDir: String) = this(logDir, Utils.getHadoopFileSystem(logDir)) private var applicationComplete = false + private var sparkVersion: Option[String] = None private var compressionCodec: Option[CompressionCodec] = None private var logPaths = Array[Path]() private var started = false @@ -52,25 +53,27 @@ private[spark] class ReplayListenerBus( * Prepare state for reading event logs. * * This gathers relevant files in the given directory and extracts meaning from each category. - * More specifically, this involves looking for event logs, the compression codec file - * (if event logs are compressed), and the application completion file (if the application - * has run to completion). + * More specifically, this involves looking for event logs, the Spark version file, the + * compression codec file (if event logs are compressed), and the application completion + * file (if the application has run to completion). */ def start() { val filePaths = getFilePaths(logDir, fileSystem) - logPaths = filePaths.filter { file => EventLoggingListener.isEventLogFile(file.getName) } - compressionCodec = - filePaths.find { file => - EventLoggingListener.isCompressionCodecFile(file.getName) - }.map { file => + logPaths = filePaths + .filter { file => EventLoggingListener.isEventLogFile(file.getName) } + sparkVersion = filePaths + .find { file => EventLoggingListener.isSparkVersionFile(file.getName) } + .map { file => EventLoggingListener.parseSparkVersion(file.getName) } + compressionCodec = filePaths + .find { file => EventLoggingListener.isCompressionCodecFile(file.getName) } + .map { file => val codec = EventLoggingListener.parseCompressionCodec(file.getName) val conf = new SparkConf conf.set("spark.io.compression.codec", codec) CompressionCodec.createCodec(conf) } - applicationComplete = filePaths.exists { file => - EventLoggingListener.isApplicationCompleteFile(file.getName) - } + applicationComplete = filePaths + .exists { file => EventLoggingListener.isApplicationCompleteFile(file.getName) } started = true } @@ -80,6 +83,12 @@ private[spark] class ReplayListenerBus( applicationComplete } + /** Return the version of Spark on which the given application was run. */ + def getSparkVersion: String = { + assert(started, "ReplayListenerBus not started yet") + sparkVersion.getOrElse("") + } + /** * Replay each event in the order maintained in the given logs. This should only be called * exactly once. Return whether event logs are actually found. From 2dfb4940b2e3c57651970741d7d30113833247de Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 9 Apr 2014 19:09:27 -0700 Subject: [PATCH 14/18] Decouple checking for application completion from replaying This involves moving a chunk of logic from ReplayListenerBus. Previously, the ReplayListenerBus also takes care of parsing logging information, which requires it to understand the format of EventLoggingListener. This leads to increased complexity if we want to check whether an application has completed independently from replaying its logs, as we do in HistoryServer. This wide refactoring also affects Master, which must now go through the same code path as the HistoryServer to parse the logging information before replaying events. --- .../spark/deploy/history/HistoryServer.scala | 124 ++++++------------ .../spark/deploy/history/IndexPage.scala | 8 +- .../apache/spark/deploy/master/Master.scala | 51 ++++--- .../scheduler/EventLoggingListener.scala | 88 +++++++++++-- .../spark/scheduler/ReplayListenerBus.scala | 99 ++------------ 5 files changed, 164 insertions(+), 206 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index b96865c95c52b..7e4833cbd759e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -26,10 +26,10 @@ import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.deploy.SparkUIContainer +import org.apache.spark.scheduler._ import org.apache.spark.ui.SparkUI import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.Utils -import org.apache.spark.scheduler.{ApplicationEventListener, ReplayListenerBus} /** * A web server that renders SparkUIs of finished applications. @@ -63,11 +63,8 @@ class HistoryServer( // A timestamp of when the disk was last accessed to check for log updates private var lastLogCheckTime = -1L - // If an application is last updated after this threshold, then its UI is retained - private var updateTimeThreshold = -1L - - // Number of applications hidden from the UI because the application limit has been reached - private var numApplicationsHidden = 0 + // Number of complete applications found in this directory + private var numApplicationsTotal = 0 @volatile private var stopped = false @@ -124,7 +121,6 @@ class HistoryServer( logError("Failed to bind HistoryServer", e) System.exit(1) } - checkForLogs() } /** @@ -145,41 +141,33 @@ class HistoryServer( try { val logStatus = fileSystem.listStatus(new Path(baseLogDir)) val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]() + val logInfos = logDirs + .sortBy { dir => getModificationTime(dir) } + .map { dir => (dir, EventLoggingListener.parseLoggingInfo(dir.getPath, fileSystem)) } + .filter { case (dir, info) => info.applicationComplete } + + // Logging information for applications that should be retained + val retainedLogInfos = logInfos.takeRight(RETAINED_APPLICATIONS) + val retainedAppIds = retainedLogInfos.map { case (dir, _) => dir.getPath.getName } - // Forget about any SparkUIs that can no longer be found - val mostRecentAppIds = logDirs.map { dir => getAppId(dir.getPath.toString) } + // Remove any applications that should no longer be retained appIdToInfo.foreach { case (appId, info) => - if (!mostRecentAppIds.contains(appId)) { + if (!retainedAppIds.contains(appId)) { detachUI(info.ui) appIdToInfo.remove(appId) - updateTimeThreshold = -1L } } - // Keep track of the number of applications hidden from the UI this round - var _numApplicationsHidden = 0 - - // Render SparkUI for any new completed applications - logDirs.foreach { dir => - val path = dir.getPath.toString - val appId = getAppId(path) - val lastUpdated = getModificationTime(dir) + // Render the application's UI if it is not already there + retainedLogInfos.foreach { case (dir, info) => + val appId = dir.getPath.getName if (!appIdToInfo.contains(appId)) { - if (lastUpdated > updateTimeThreshold) { - maybeRenderUI(appId, path, lastUpdated) - } else { - // This application was previously blacklisted due to the application limit - _numApplicationsHidden += 1 - } - } - // If the cap is reached, remove the least recently updated application - if (appIdToInfo.size > RETAINED_APPLICATIONS) { - removeOldestApp() - _numApplicationsHidden += 1 + renderSparkUI(dir, info) } } - numApplicationsHidden = _numApplicationsHidden + // Track the total number of complete applications observed this round + numApplicationsTotal = logInfos.size } catch { case t: Throwable => logError("Exception in checking for event log updates", t) @@ -196,51 +184,27 @@ class HistoryServer( * directory. If this file exists, the associated application is regarded to be complete, in * which case the server proceeds to render the SparkUI. Otherwise, the server does nothing. */ - private def maybeRenderUI(appId: String, logPath: String, lastUpdated: Long) { - val replayBus = new ReplayListenerBus(logPath, fileSystem) - replayBus.start() - - // If the application completion file is found - if (replayBus.isApplicationComplete) { - val ui = new SparkUI(replayBus, appId, "/history/%s".format(appId)) - val appListener = new ApplicationEventListener - replayBus.addListener(appListener) - - // Do not call ui.bind() to avoid creating a new server for each application - ui.start() - val success = replayBus.replay() - if (success && appListener.applicationStarted) { - attachUI(ui) - val appName = appListener.appName - val sparkUser = appListener.sparkUser - val startTime = appListener.startTime - val endTime = appListener.endTime - ui.setAppName("%s (finished)".format(appName)) - appIdToInfo(appId) = - ApplicationHistoryInfo(appName, startTime, endTime, lastUpdated, sparkUser, logPath, ui) - } else { - logWarning("Reconstructing application UI was unsuccessful. Either no event logs were" + - "found or the event signaling application start is missing: %s".format(logPath)) - } - } else { - logWarning("Skipping incomplete application: %s".format(logPath)) - } - } - - /** - * Remove the oldest application and detach its associated UI. - * - * As an optimization, record the last updated time of this application as the minimum - * update time threshold. Only applications with a last updated time that exceeds this - * threshold will be retained by the server. This avoids re-rendering an old application - * that is recently removed. - */ - private def removeOldestApp() { - val appToRemove = appIdToInfo.toSeq.minBy { case (_, info) => info.lastUpdated } - appToRemove match { case (id, info) => - appIdToInfo.remove(id) - detachUI(info.ui) - updateTimeThreshold = info.lastUpdated + private def renderSparkUI(logDir: FileStatus, logInfo: EventLoggingInfo) { + val path = logDir.getPath + val appId = path.getName + val replayBus = new ReplayListenerBus(logInfo.logPaths, fileSystem, logInfo.compressionCodec) + val ui = new SparkUI(replayBus, appId, "/history/" + appId) + val appListener = new ApplicationEventListener + replayBus.addListener(appListener) + + // Do not call ui.bind() to avoid creating a new server for each application + ui.start() + replayBus.replay() + if (appListener.applicationStarted) { + attachUI(ui) + val appName = appListener.appName + val sparkUser = appListener.sparkUser + val startTime = appListener.startTime + val endTime = appListener.endTime + val lastUpdated = getModificationTime(logDir) + ui.setAppName(appName + " (finished)") + appIdToInfo(appId) = ApplicationHistoryInfo(appId, appName, startTime, endTime, + lastUpdated, sparkUser, path, ui) } } @@ -251,14 +215,11 @@ class HistoryServer( fileSystem.close() } - /** Parse app ID from the given log path. */ - def getAppId(logPath: String): String = logPath.split("/").last - /** Return the address of this server. */ def getAddress: String = "http://" + publicHost + ":" + boundPort /** Return the total number of application logs found, whether or not the UI is retained. */ - def getTotalApplications: Int = appIdToInfo.size + numApplicationsHidden + def getNumApplications: Int = numApplicationsTotal /** Return when this directory was last modified. */ private def getModificationTime(dir: FileStatus): Long = { @@ -312,12 +273,13 @@ object HistoryServer { private[spark] case class ApplicationHistoryInfo( + id: String, name: String, startTime: Long, endTime: Long, lastUpdated: Long, sparkUser: String, - logPath: String, + logDirPath: Path, ui: SparkUI) { def started = startTime != -1 def finished = endTime != -1 diff --git a/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala index e66463eef08c6..875f4e1e4bafc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala @@ -37,8 +37,8 @@ private[spark] class IndexPage(parent: HistoryServer) { { if (parent.appIdToInfo.size > 0) {

- Showing {parent.appIdToInfo.size}/{parent.getTotalApplications} - Finished Application{if (parent.getTotalApplications > 1) "s" else ""} + Showing {parent.appIdToInfo.size}/{parent.getNumApplications} + Finished Application{if (parent.getNumApplications > 1) "s" else ""}

++ appTable } else { @@ -60,14 +60,14 @@ private[spark] class IndexPage(parent: HistoryServer) { "Last Updated") private def appRow(info: ApplicationHistoryInfo): Seq[Node] = { - val appName = if (info.started) info.name else parent.getAppId(info.logPath) + val appName = if (info.started) info.name else info.logDirPath.getName val uiAddress = parent.getAddress + info.ui.basePath val startTime = if (info.started) WebUI.formatDate(info.startTime) else "Not started" val endTime = if (info.finished) WebUI.formatDate(info.endTime) else "Not finished" val difference = if (info.started && info.finished) info.endTime - info.startTime else -1L val duration = if (difference > 0) WebUI.formatDuration(difference) else "---" val sparkUser = if (info.started) info.sparkUser else "Unknown user" - val logDirectory = parent.getAppId(info.logPath) + val logDirectory = info.logDirPath.getName val lastUpdated = WebUI.formatDate(info.lastUpdated) {appName} 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 d940bc44569fd..ca0dd7552fc2f 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 @@ -38,7 +38,7 @@ 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.scheduler.{EventLoggingListener, ReplayListenerBus} import org.apache.spark.ui.SparkUI import org.apache.spark.util.{AkkaUtils, Utils} @@ -73,7 +73,7 @@ private[spark] class Master( var nextAppNumber = 0 val appIdToUI = new HashMap[String, SparkUI] - val fileSystems = new HashSet[FileSystem] + val fileSystemsUsed = new HashSet[FileSystem] val drivers = new HashSet[DriverInfo] val completedDrivers = new ArrayBuffer[DriverInfo] @@ -152,7 +152,7 @@ private[spark] class Master( override def postStop() { webUi.stop() - fileSystems.foreach(_.close()) + fileSystemsUsed.foreach(_.close()) masterMetricsSystem.stop() applicationMetricsSystem.stop() persistenceEngine.close() @@ -634,11 +634,7 @@ private[spark] class Master( 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 { + if (!rebuildSparkUI(app)) { // Avoid broken links if the UI is not reconstructed app.desc.appUiUrl = "" } @@ -658,21 +654,34 @@ private[spark] class Master( } /** - * Start a new SparkUI rendered from persisted storage. If this is unsuccessful for any reason, - * return None. Otherwise return the reconstructed UI. + * Rebuild a new SparkUI from the given application's event logs. + * Return whether this is successful. */ - def startPersistedSparkUI(app: ApplicationInfo): Option[SparkUI] = { + def rebuildSparkUI(app: ApplicationInfo): Boolean = { val appName = app.desc.name - val eventLogDir = app.desc.eventLogDir.getOrElse { return None } - val replayBus = new ReplayListenerBus(eventLogDir) - val ui = new SparkUI(replayBus, "%s (finished)".format(appName), "/history/%s".format(app.id)) - fileSystems += replayBus.fileSystem - - // Do not call ui.bind() to avoid creating a new server for each application - ui.start() - replayBus.start() - val success = replayBus.replay() - if (success) Some(ui) else None + val eventLogDir = app.desc.eventLogDir.getOrElse { return false } + val fileSystem = Utils.getHadoopFileSystem(eventLogDir) + val eventLogInfo = EventLoggingListener.parseLoggingInfo(eventLogDir, fileSystem) + val eventLogPaths = eventLogInfo.logPaths + val compressionCodec = eventLogInfo.compressionCodec + if (!eventLogPaths.isEmpty) { + try { + val replayBus = new ReplayListenerBus(eventLogPaths, fileSystem, compressionCodec) + val ui = new SparkUI(replayBus, appName + " (finished)", "/history/" + app.id) + ui.start() + replayBus.replay() + app.desc.appUiUrl = ui.basePath + appIdToUI(app.id) = ui + webUi.attachUI(ui) + return true + } catch { + case t: Throwable => + logError("Exception in replaying log for application %s (%s)".format(appName, app.id), t) + } + } else { + logWarning("Application %s (%s) has no valid logs: %s".format(appName, app.id, eventLogDir)) + } + false } /** Generate a new app ID given a app's submission date */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index c3173e2ab0859..a642ebac4df97 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -17,11 +17,12 @@ package org.apache.spark.scheduler +import org.apache.hadoop.fs.{FileSystem, Path} import org.json4s.jackson.JsonMethods._ import org.apache.spark.{Logging, SparkConf, SparkContext} import org.apache.spark.io.CompressionCodec -import org.apache.spark.util.{JsonProtocol, FileLogger} +import org.apache.spark.util.{FileLogger, JsonProtocol} /** * A SparkListener that logs events to persistent storage. @@ -49,8 +50,8 @@ private[spark] class EventLoggingListener(appName: String, conf: SparkConf) new FileLogger(logDir, conf, outputBufferSize, shouldCompress, shouldOverwrite) /** - * Begin logging events. If compression is used, log a file that indicates which compression - * library is used. + * Begin logging events. + * If compression is used, log a file that indicates which compression library is used. */ def start() { logInfo("Logging events to %s".format(logDir)) @@ -62,7 +63,7 @@ private[spark] class EventLoggingListener(appName: String, conf: SparkConf) logger.newFile(LOG_PREFIX + logger.fileIndex) } - /** Log the event as JSON */ + /** Log the event as JSON. */ private def logEvent(event: SparkListenerEvent, flushLogger: Boolean = false) { val eventJson = compact(render(JsonProtocol.sparkEventToJson(event))) logger.logLine(eventJson) @@ -102,8 +103,8 @@ private[spark] class EventLoggingListener(appName: String, conf: SparkConf) logEvent(event, flushLogger = true) /** - * Stop logging events. In addition, create an empty special file to indicate application - * completion. + * Stop logging events. + * In addition, create an empty special file to indicate application completion. */ def stop() = { logger.newFile(APPLICATION_COMPLETE) @@ -111,20 +112,20 @@ private[spark] class EventLoggingListener(appName: String, conf: SparkConf) } } -private[spark] object EventLoggingListener { - val SPARK_VERSION_PREFIX = "SPARK_VERSION_" +private[spark] object EventLoggingListener extends Logging { val LOG_PREFIX = "EVENT_LOG_" + val SPARK_VERSION_PREFIX = "SPARK_VERSION_" val COMPRESSION_CODEC_PREFIX = "COMPRESSION_CODEC_" val APPLICATION_COMPLETE = "APPLICATION_COMPLETE" - def isSparkVersionFile(fileName: String): Boolean = { - fileName.startsWith(SPARK_VERSION_PREFIX) - } - def isEventLogFile(fileName: String): Boolean = { fileName.startsWith(LOG_PREFIX) } + def isSparkVersionFile(fileName: String): Boolean = { + fileName.startsWith(SPARK_VERSION_PREFIX) + } + def isCompressionCodecFile(fileName: String): Boolean = { fileName.startsWith(COMPRESSION_CODEC_PREFIX) } @@ -144,4 +145,67 @@ private[spark] object EventLoggingListener { fileName.replaceAll(COMPRESSION_CODEC_PREFIX, "") } else "" } + + /** + * Parse the event logging information associated with the logs in the given directory. + * + * Specifically, this looks for event log files, the Spark version file, the compression + * codec file (if event logs are compressed), and the application completion file (if the + * application has run to completion). + */ + def parseLoggingInfo(logDir: Path, fileSystem: FileSystem): EventLoggingInfo = { + try { + val fileStatuses = fileSystem.listStatus(logDir) + val filePaths = + if (fileStatuses != null) { + fileStatuses.filter(!_.isDir).map(_.getPath).toSeq + } else { + Seq[Path]() + } + if (filePaths.isEmpty) { + logWarning("No files found in logging directory %s".format(logDir)) + } + EventLoggingInfo( + logPaths = filePaths.filter { path => isEventLogFile(path.getName) }, + sparkVersion = filePaths + .find { path => isSparkVersionFile(path.getName) } + .map { path => parseSparkVersion(path.getName) } + .getOrElse(""), + compressionCodec = filePaths + .find { path => isCompressionCodecFile(path.getName) } + .map { path => + val codec = EventLoggingListener.parseCompressionCodec(path.getName) + val conf = new SparkConf + conf.set("spark.io.compression.codec", codec) + CompressionCodec.createCodec(conf) + }, + applicationComplete = filePaths.exists { path => isApplicationCompleteFile(path.getName) } + ) + } catch { + case t: Throwable => + logError("Exception in parsing logging info from directory %s".format(logDir), t) + EventLoggingInfo.empty + } + } + + /** + * Parse the event logging information associated with the logs in the given directory. + */ + def parseLoggingInfo(logDir: String, fileSystem: FileSystem): EventLoggingInfo = { + parseLoggingInfo(new Path(logDir), fileSystem) + } +} + + +/** + * Information needed to process the event logs associated with an application. + */ +private[spark] case class EventLoggingInfo( + logPaths: Seq[Path], + sparkVersion: String, + compressionCodec: Option[CompressionCodec], + applicationComplete: Boolean = false) + +private[spark] object EventLoggingInfo { + def empty = EventLoggingInfo(Seq[Path](), "", None, applicationComplete = false) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala index 617e956a56c54..b03665fd56d33 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -25,9 +25,9 @@ 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.Logging import org.apache.spark.io.CompressionCodec -import org.apache.spark.util.{JsonProtocol, Utils} +import org.apache.spark.util.JsonProtocol /** * A SparkListenerBus that replays logged events from persisted storage. @@ -36,72 +36,23 @@ import org.apache.spark.util.{JsonProtocol, Utils} * There exists a one-to-one mapping between ReplayListenerBus and event logging applications. */ private[spark] class ReplayListenerBus( - logDir: String, - val fileSystem: FileSystem) + logPaths: Seq[Path], + fileSystem: FileSystem, + compressionCodec: Option[CompressionCodec]) extends SparkListenerBus with Logging { - def this(logDir: String) = this(logDir, Utils.getHadoopFileSystem(logDir)) - - private var applicationComplete = false - private var sparkVersion: Option[String] = None - private var compressionCodec: Option[CompressionCodec] = None - private var logPaths = Array[Path]() - private var started = false private var replayed = false - /** - * Prepare state for reading event logs. - * - * This gathers relevant files in the given directory and extracts meaning from each category. - * More specifically, this involves looking for event logs, the Spark version file, the - * compression codec file (if event logs are compressed), and the application completion - * file (if the application has run to completion). - */ - def start() { - val filePaths = getFilePaths(logDir, fileSystem) - logPaths = filePaths - .filter { file => EventLoggingListener.isEventLogFile(file.getName) } - sparkVersion = filePaths - .find { file => EventLoggingListener.isSparkVersionFile(file.getName) } - .map { file => EventLoggingListener.parseSparkVersion(file.getName) } - compressionCodec = filePaths - .find { file => EventLoggingListener.isCompressionCodecFile(file.getName) } - .map { file => - val codec = EventLoggingListener.parseCompressionCodec(file.getName) - val conf = new SparkConf - conf.set("spark.io.compression.codec", codec) - CompressionCodec.createCodec(conf) - } - applicationComplete = filePaths - .exists { file => EventLoggingListener.isApplicationCompleteFile(file.getName) } - started = true - } - - /** Return whether the associated application signaled completion. */ - def isApplicationComplete: Boolean = { - assert(started, "ReplayListenerBus not started yet") - applicationComplete - } - - /** Return the version of Spark on which the given application was run. */ - def getSparkVersion: String = { - assert(started, "ReplayListenerBus not started yet") - sparkVersion.getOrElse("") + if (logPaths.length == 0) { + logWarning("Log path provided contains no log files.") } /** - * Replay each event in the order maintained in the given logs. This should only be called - * exactly once. Return whether event logs are actually found. + * Replay each event in the order maintained in the given logs. + * This should only be called exactly once. */ - def replay(): Boolean = { - assert(started, "ReplayListenerBus must be started before replaying logged events") + def replay() { assert(!replayed, "ReplayListenerBus cannot replay events more than once") - - if (logPaths.length == 0) { - logWarning("Log path provided contains no log files: %s".format(logDir)) - 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 @@ -114,7 +65,7 @@ private[spark] class ReplayListenerBus( bufferedStream = Some(new FastBufferedInputStream(fileStream.get)) compressStream = Some(wrapForCompression(bufferedStream.get)) - // Parse each line as an event and post it to all attached listeners + // Parse each line as an event and post the event to all attached listeners val lines = Source.fromInputStream(compressStream.get).getLines() lines.foreach { line => currentLine = line @@ -130,39 +81,11 @@ private[spark] class ReplayListenerBus( compressStream.foreach(_.close()) } } - replayed = true - true - } - - /** Stop the file system. */ - def stop() { - fileSystem.close() } /** If a compression codec is specified, wrap the given stream in a compression stream. */ private def wrapForCompression(stream: InputStream): InputStream = { compressionCodec.map(_.compressedInputStream(stream)).getOrElse(stream) } - - /** Return a list of paths representing files found in the given directory. */ - private def getFilePaths(logDir: String, fileSystem: FileSystem): Array[Path] = { - try { - 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("No files are found in the given log directory: %s".format(logDir)) - return Array[Path]() - } - logStatus.filter(!_.isDir).map(_.getPath).sortBy(_.getName) - } catch { - case t: Throwable => - logError("Exception in accessing log files in %s".format(logDir), t) - Array[Path]() - } - } } From f7f5bf034177df4e608cbbf4b9b52585ece007fd Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 9 Apr 2014 19:27:09 -0700 Subject: [PATCH 15/18] Make history server's web UI port a Spark configuration --- .../spark/deploy/history/HistoryServer.scala | 11 ++++++----- .../history/HistoryServerArguments.scala | 10 ++-------- .../spark/deploy/worker/ui/WorkerWebUI.scala | 4 ++-- .../scala/org/apache/spark/ui/SparkUI.scala | 4 ++-- docs/monitoring.md | 18 ++++++++++++------ sbin/start-history-server.sh | 15 +++------------ 6 files changed, 27 insertions(+), 35 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 7e4833cbd759e..3ba04912b0c88 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -43,11 +43,9 @@ import org.apache.spark.util.Utils * EventLoggingListener. * * @param baseLogDir The base directory in which event logs are found - * @param requestedPort The requested port to which this server is to be bound */ class HistoryServer( val baseLogDir: String, - requestedPort: Int, conf: SparkConf) extends SparkUIContainer("History Server") with Logging { @@ -56,7 +54,7 @@ class HistoryServer( private val fileSystem = Utils.getHadoopFileSystem(baseLogDir) private val localHost = Utils.localHostName() private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHost) - private val port = requestedPort + private val port = WEB_UI_PORT private val securityManager = new SecurityManager(conf) private val indexPage = new IndexPage(this) @@ -243,7 +241,7 @@ class HistoryServer( * start-history-server.sh and stop-history-server.sh. The path to a base log directory * is must be specified, while the requested UI port is optional. For example: * - * ./sbin/spark-history-server.sh /tmp/spark-events 18080 + * ./sbin/spark-history-server.sh /tmp/spark-events * ./sbin/spark-history-server.sh hdfs://1.2.3.4:9000/spark-events * * This launches the HistoryServer as a Spark daemon. @@ -257,11 +255,14 @@ object HistoryServer { // How many applications to retain val RETAINED_APPLICATIONS = conf.getInt("spark.history.retainedApplications", 250) + // The port to which the web UI is bound + val WEB_UI_PORT = conf.getInt("spark.history.ui.port", 18080) + val STATIC_RESOURCE_DIR = SparkUI.STATIC_RESOURCE_DIR def main(argStrings: Array[String]) { val args = new HistoryServerArguments(argStrings) - val server = new HistoryServer(args.logDir, args.port, conf) + val server = new HistoryServer(args.logDir, conf) server.bind() server.start() diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala index d221ad32bb048..f5d21e768b38f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala @@ -21,23 +21,18 @@ import java.net.URI import org.apache.hadoop.fs.Path -import org.apache.spark.util.{IntParam, Utils} +import org.apache.spark.util.Utils /** * Command-line parser for the master. */ private[spark] class HistoryServerArguments(args: Array[String]) { - var port = 18080 var logDir = "" parse(args.toList) private def parse(args: List[String]): Unit = { args match { - case ("--port" | "-p") :: IntParam(value) :: tail => - port = value - parse(tail) - case ("--dir" | "-d") :: value :: tail => logDir = value parse(tail) @@ -45,7 +40,7 @@ private[spark] class HistoryServerArguments(args: Array[String]) { case ("--help" | "-h") :: tail => printUsageAndExit(0) - case Nil => {} + case Nil => case _ => printUsageAndExit(1) @@ -71,7 +66,6 @@ private[spark] class HistoryServerArguments(args: Array[String]) { "Usage: HistoryServer [options]\n" + "\n" + "Options:\n" + - " -p PORT, --port PORT Port for web server (default: 18080)\n" + " -d DIR, --dir DIR Location of event log files") System.exit(exitCode) } 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 f9d6436884025..5625a44549aaa 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 @@ -39,7 +39,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I private val host = Utils.localHostName() private val port = requestedPort.getOrElse( - worker.conf.get("worker.ui.port", WorkerWebUI.DEFAULT_PORT).toInt) + worker.conf.getInt("worker.ui.port", WorkerWebUI.DEFAULT_PORT)) private val indexPage = new IndexPage(this) private val handlers: Seq[ServletContextHandler] = { @@ -188,6 +188,6 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I } private[spark] object WorkerWebUI { + val DEFAULT_PORT=8081 val STATIC_RESOURCE_BASE = SparkUI.STATIC_RESOURCE_DIR - val DEFAULT_PORT="8081" } 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 9895ead469fbc..b8e6e15880bf5 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -49,7 +49,7 @@ private[spark] class SparkUI( private val localHost = Utils.localHostName() private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHost) - private val port = conf.get("spark.ui.port", SparkUI.DEFAULT_PORT).toInt + private val port = conf.getInt("spark.ui.port", SparkUI.DEFAULT_PORT) private val storage = new BlockManagerUI(this) private val jobs = new JobProgressUI(this) @@ -118,6 +118,6 @@ private[spark] class SparkUI( } private[spark] object SparkUI { - val DEFAULT_PORT = "4040" + val DEFAULT_PORT = 4040 val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static" } diff --git a/docs/monitoring.md b/docs/monitoring.md index ee7854ccd66fd..4c91c3a5929bf 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -39,8 +39,7 @@ You can start a the history server by executing: The base logging directory must be supplied, and should contain sub-directories that each represents an application's event logs. This creates a web interface at -`http://:18080` by default, but the port can be changed by supplying an extra -parameter to the start script. The history server depends on the following variables: +`http://:18080` by default. The history server depends on the following variables: @@ -62,16 +61,23 @@ Further, the history server can be configured as follows: + + + + +
Environment VariableMeaning
spark.history.updateInterval 10 - The period at which information displayed by this history server is updated. Each update - checks for any changes made to the event logs in persisted storage. + The period, in seconds, at which information displayed by this history server is updated. + Each update checks for any changes made to the event logs in persisted storage.
spark.history.retainedApplications 250 - The number of application UIs to retain. If this cap is exceeded, then the least recently - updated applications will be removed. + The number of application UIs to retain. If this cap is exceeded, then the oldest + applications will be removed. +
spark.history.ui.port18080 + The port to which the web interface of the history server binds.
diff --git a/sbin/start-history-server.sh b/sbin/start-history-server.sh index 76ca799862e00..4a90c68763b68 100755 --- a/sbin/start-history-server.sh +++ b/sbin/start-history-server.sh @@ -27,20 +27,11 @@ sbin=`dirname "$0"` sbin=`cd "$sbin"; pwd` if [ $# -lt 1 ]; then - echo "Usage: ./start-history-server.sh []" - echo "Example: ./start-history-server.sh /tmp/spark-events 18080" + echo "Usage: ./start-history-server.sh " + echo "Example: ./start-history-server.sh /tmp/spark-events" exit fi -# Set up base event log directory LOG_DIR=$1 -shift -# Set up web UI port -if [ ! -z $1 ]; then - PORT=$1 -else - PORT=18080 -fi - -"$sbin"/spark-daemon.sh start org.apache.spark.deploy.history.HistoryServer 1 --dir "$LOG_DIR" --port "$PORT" +"$sbin"/spark-daemon.sh start org.apache.spark.deploy.history.HistoryServer 1 --dir "$LOG_DIR" From 69d1b4141f08f246a78728f0e4a57b46d80ea6ca Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 9 Apr 2014 20:12:26 -0700 Subject: [PATCH 16/18] Do not block on posting SparkListenerApplicationEnd --- .../scala/org/apache/spark/SparkContext.scala | 2 +- .../spark/scheduler/LiveListenerBus.scala | 19 +++++++------------ 2 files changed, 8 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 1d79bed6cc0ee..e6c9b7000d819 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1188,7 +1188,7 @@ class SparkContext(config: SparkConf) extends Logging { * condition exists in which the listeners may stop before this event has been propagated. */ private def postApplicationEnd() { - listenerBus.post(SparkListenerApplicationEnd(System.currentTimeMillis), blocking = true) + listenerBus.post(SparkListenerApplicationEnd(System.currentTimeMillis)) } /** Post the environment update event once the task scheduler is ready */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala index ef13b26a6c047..76f3e327d60b8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala @@ -68,18 +68,13 @@ private[spark] class LiveListenerBus extends SparkListenerBus with Logging { started = true } - def post(event: SparkListenerEvent, blocking: Boolean = false) { - if (!blocking) { - 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 - } - } else { - // Bypass the event queue and post to all attached listeners immediately - postToAll(event) + 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 } } From b158d985a87f5c44beaf51d7342908fce3843f57 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 9 Apr 2014 22:21:22 -0700 Subject: [PATCH 17/18] Address Patrick's comments --- .../spark/deploy/history/HistoryServer.scala | 28 +++++++++---------- .../history/HistoryServerArguments.scala | 8 ++++-- .../spark/deploy/history/IndexPage.scala | 4 +-- .../apache/spark/deploy/master/Master.scala | 2 +- .../scheduler/ApplicationEventListener.scala | 5 ++-- .../scheduler/EventLoggingListener.scala | 15 ++++++---- 6 files changed, 35 insertions(+), 27 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 3ba04912b0c88..97d2ba9deed33 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -32,7 +32,7 @@ import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.Utils /** - * A web server that renders SparkUIs of finished applications. + * A web server that renders SparkUIs of completed applications. * * For the standalone mode, MasterWebUI already achieves this functionality. Thus, the * main use case of the HistoryServer is in other deploy modes (e.g. Yarn or Mesos). @@ -61,8 +61,8 @@ class HistoryServer( // A timestamp of when the disk was last accessed to check for log updates private var lastLogCheckTime = -1L - // Number of complete applications found in this directory - private var numApplicationsTotal = 0 + // Number of completed applications found in this directory + private var numCompletedApplications = 0 @volatile private var stopped = false @@ -125,11 +125,11 @@ class HistoryServer( * Check for any updates to event logs in the base directory. This is only effective once * the server has been bound. * - * If a new finished application is found, the server renders the associated SparkUI + * If a new completed application is found, the server renders the associated SparkUI * from the application's event logs, attaches this UI to itself, and stores metadata * information for this application. * - * If the logs for an existing finished application are no longer found, the server + * If the logs for an existing completed application are no longer found, the server * removes all associated information and detaches the SparkUI. */ def checkForLogs() = synchronized { @@ -164,8 +164,8 @@ class HistoryServer( } } - // Track the total number of complete applications observed this round - numApplicationsTotal = logInfos.size + // Track the total number of completed applications observed this round + numCompletedApplications = logInfos.size } catch { case t: Throwable => logError("Exception in checking for event log updates", t) @@ -176,10 +176,10 @@ class HistoryServer( } /** - * Render a new SparkUI from the event logs if the associated application is finished. + * Render a new SparkUI from the event logs if the associated application is completed. * * HistoryServer looks for a special file that indicates application completion in the given - * directory. If this file exists, the associated application is regarded to be complete, in + * directory. If this file exists, the associated application is regarded to be completed, in * which case the server proceeds to render the SparkUI. Otherwise, the server does nothing. */ private def renderSparkUI(logDir: FileStatus, logInfo: EventLoggingInfo) { @@ -200,7 +200,7 @@ class HistoryServer( val startTime = appListener.startTime val endTime = appListener.endTime val lastUpdated = getModificationTime(logDir) - ui.setAppName(appName + " (finished)") + ui.setAppName(appName + " (completed)") appIdToInfo(appId) = ApplicationHistoryInfo(appId, appName, startTime, endTime, lastUpdated, sparkUser, path, ui) } @@ -216,14 +216,14 @@ class HistoryServer( /** Return the address of this server. */ def getAddress: String = "http://" + publicHost + ":" + boundPort - /** Return the total number of application logs found, whether or not the UI is retained. */ - def getNumApplications: Int = numApplicationsTotal + /** Return the number of completed applications found, whether or not the UI is rendered. */ + def getNumApplications: Int = numCompletedApplications /** Return when this directory was last modified. */ private def getModificationTime(dir: FileStatus): Long = { try { val logFiles = fileSystem.listStatus(dir.getPath) - if (logFiles != null) { + if (logFiles != null && !logFiles.isEmpty) { logFiles.map(_.getModificationTime).max } else { dir.getModificationTime @@ -283,5 +283,5 @@ private[spark] case class ApplicationHistoryInfo( logDirPath: Path, ui: SparkUI) { def started = startTime != -1 - def finished = endTime != -1 + def completed = endTime != -1 } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala index f5d21e768b38f..943c061743dbd 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala @@ -55,8 +55,12 @@ private[spark] class HistoryServerArguments(args: Array[String]) { } val fileSystem = Utils.getHadoopFileSystem(new URI(logDir)) val path = new Path(logDir) - if (!fileSystem.exists(path) || !fileSystem.getFileStatus(path).isDir) { - System.err.println("Logging directory specified is invalid: %s".format(logDir)) + if (!fileSystem.exists(path)) { + System.err.println("Logging directory specified does not exist: %s".format(logDir)) + printUsageAndExit(1) + } + if (!fileSystem.getFileStatus(path).isDir) { + System.err.println("Logging directory specified is not a directory: %s".format(logDir)) printUsageAndExit(1) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala index 875f4e1e4bafc..4d96e2b7b6f70 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala @@ -63,8 +63,8 @@ private[spark] class IndexPage(parent: HistoryServer) { val appName = if (info.started) info.name else info.logDirPath.getName val uiAddress = parent.getAddress + info.ui.basePath val startTime = if (info.started) WebUI.formatDate(info.startTime) else "Not started" - val endTime = if (info.finished) WebUI.formatDate(info.endTime) else "Not finished" - val difference = if (info.started && info.finished) info.endTime - info.startTime else -1L + val endTime = if (info.completed) WebUI.formatDate(info.endTime) else "Not finished" + val difference = if (info.started && info.completed) info.endTime - info.startTime else -1L val duration = if (difference > 0) WebUI.formatDuration(difference) else "---" val sparkUser = if (info.started) info.sparkUser else "Unknown user" val logDirectory = info.logDirPath.getName 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 ca0dd7552fc2f..2446e86cb6672 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 @@ -667,7 +667,7 @@ private[spark] class Master( if (!eventLogPaths.isEmpty) { try { val replayBus = new ReplayListenerBus(eventLogPaths, fileSystem, compressionCodec) - val ui = new SparkUI(replayBus, appName + " (finished)", "/history/" + app.id) + val ui = new SparkUI(replayBus, appName + " (completed)", "/history/" + app.id) ui.start() replayBus.replay() app.desc.appUiUrl = ui.basePath diff --git a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala index 9f630e20e0b09..affda13df6531 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala @@ -20,9 +20,8 @@ package org.apache.spark.scheduler /** * A simple listener for application events. * - * This listener assumes at most one of each of SparkListenerApplicationStart and - * SparkListenerApplicationEnd will be received. Otherwise, only the latest event - * of each type will take effect. + * This listener expects to hear events from a single application only. If events + * from multiple applications are seen, the behavior is unspecified. */ private[spark] class ApplicationEventListener extends SparkListener { var appName = "" diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index a642ebac4df97..b983c16af14f4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -17,6 +17,8 @@ package org.apache.spark.scheduler +import scala.collection.mutable + import org.apache.hadoop.fs.{FileSystem, Path} import org.json4s.jackson.JsonMethods._ @@ -118,6 +120,9 @@ private[spark] object EventLoggingListener extends Logging { val COMPRESSION_CODEC_PREFIX = "COMPRESSION_CODEC_" val APPLICATION_COMPLETE = "APPLICATION_COMPLETE" + // A cache for compression codecs to avoid creating the same codec many times + private val codecMap = new mutable.HashMap[String, CompressionCodec] + def isEventLogFile(fileName: String): Boolean = { fileName.startsWith(LOG_PREFIX) } @@ -174,11 +179,11 @@ private[spark] object EventLoggingListener extends Logging { compressionCodec = filePaths .find { path => isCompressionCodecFile(path.getName) } .map { path => - val codec = EventLoggingListener.parseCompressionCodec(path.getName) - val conf = new SparkConf - conf.set("spark.io.compression.codec", codec) - CompressionCodec.createCodec(conf) - }, + val codec = EventLoggingListener.parseCompressionCodec(path.getName) + val conf = new SparkConf + conf.set("spark.io.compression.codec", codec) + codecMap.getOrElseUpdate(codec, CompressionCodec.createCodec(conf)) + }, applicationComplete = filePaths.exists { path => isApplicationCompleteFile(path.getName) } ) } catch { From 7b7234cd275628a4fa1ce8e2b06392f546964339 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 9 Apr 2014 23:11:58 -0700 Subject: [PATCH 18/18] Finished -> Completed --- .../scala/org/apache/spark/deploy/history/IndexPage.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala index 4d96e2b7b6f70..54dffffec71c5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala @@ -38,11 +38,11 @@ private[spark] class IndexPage(parent: HistoryServer) { if (parent.appIdToInfo.size > 0) {

Showing {parent.appIdToInfo.size}/{parent.getNumApplications} - Finished Application{if (parent.getNumApplications > 1) "s" else ""} + Completed Application{if (parent.getNumApplications > 1) "s" else ""}

++ appTable } else { -

No Finished Applications Found

+

No Completed Applications Found

} } @@ -53,7 +53,7 @@ private[spark] class IndexPage(parent: HistoryServer) { private val appHeader = Seq( "App Name", "Started", - "Finished", + "Completed", "Duration", "Spark User", "Log Directory", @@ -63,7 +63,7 @@ private[spark] class IndexPage(parent: HistoryServer) { val appName = if (info.started) info.name else info.logDirPath.getName val uiAddress = parent.getAddress + info.ui.basePath val startTime = if (info.started) WebUI.formatDate(info.startTime) else "Not started" - val endTime = if (info.completed) WebUI.formatDate(info.endTime) else "Not finished" + val endTime = if (info.completed) WebUI.formatDate(info.endTime) else "Not completed" val difference = if (info.started && info.completed) info.endTime - info.startTime else -1L val duration = if (difference > 0) WebUI.formatDuration(difference) else "---" val sparkUser = if (info.started) info.sparkUser else "Unknown user"