You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by pw...@apache.org on 2014/07/18 04:46:10 UTC

git commit: [SPARK-2411] Add a history-not-found page to standalone Master

Repository: spark
Updated Branches:
  refs/heads/master 72e9021ea -> 6afca2d10


[SPARK-2411] Add a history-not-found page to standalone Master

**Problem.** Right now, if you click on an application after it has finished, it simply refreshes the page if there are no event logs for the application. This is not super intuitive especially because event logging is not enabled by default. We should direct the user to enable this if they attempt to view a SparkUI after the fact without event logs.

**Fix.** The new page conveys different messages in each of the following scenarios:
(1) Application did not enable event logging,
(2) Event logs are not found in the specified directory, and
(3) Exception is thrown while replaying the logs

Here are screenshots of what the page looks like in each of the above scenarios:

(1)
<img src="https://issues.apache.org/jira/secure/attachment/12656204/Event%20logging%20not%20enabled.png" width="75%">

(2)
<img src="https://issues.apache.org/jira/secure/attachment/12656203/Application%20history%20not%20found.png">

(3)
<img src="https://issues.apache.org/jira/secure/attachment/12656202/Application%20history%20load%20error.png" width="95%">

Author: Andrew Or <an...@gmail.com>

Closes #1336 from andrewor14/master-link and squashes the following commits:

2f06206 [Andrew Or] Merge branch 'master' of github.com:apache/spark into master-link
97cddc0 [Andrew Or] Add different severity levels
832b687 [Andrew Or] Mention spark.eventLog.dir in error message
51980c3 [Andrew Or] Merge branch 'master' of github.com:apache/spark into master-link
ded208c [Andrew Or] Merge branch 'master' of github.com:apache/spark into master-link
89d6405 [Andrew Or] Reword message
e7df7ed [Andrew Or] Add a history not found page to standalone Master


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/6afca2d1
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/6afca2d1
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/6afca2d1

Branch: refs/heads/master
Commit: 6afca2d1079bac6309a595b8e0ffc74ae93fa662
Parents: 72e9021
Author: Andrew Or <an...@gmail.com>
Authored: Thu Jul 17 19:45:59 2014 -0700
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Thu Jul 17 19:45:59 2014 -0700

----------------------------------------------------------------------
 .../scala/org/apache/spark/TaskEndReason.scala  |  7 +-
 .../org/apache/spark/deploy/master/Master.scala | 61 ++++++++++------
 .../deploy/master/ui/ApplicationPage.scala      |  2 +-
 .../deploy/master/ui/HistoryNotFoundPage.scala  | 73 ++++++++++++++++++++
 .../spark/deploy/master/ui/MasterWebUI.scala    |  3 +-
 .../scala/org/apache/spark/util/Utils.scala     | 17 ++++-
 6 files changed, 132 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/6afca2d1/core/src/main/scala/org/apache/spark/TaskEndReason.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala
index 8d5c456..8f0c5e7 100644
--- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala
+++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala
@@ -20,6 +20,7 @@ package org.apache.spark
 import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.executor.TaskMetrics
 import org.apache.spark.storage.BlockManagerId
+import org.apache.spark.util.Utils
 
 /**
  * :: DeveloperApi ::
@@ -88,11 +89,7 @@ case class ExceptionFailure(
     stackTrace: Array[StackTraceElement],
     metrics: Option[TaskMetrics])
   extends TaskFailedReason {
-  override def toErrorString: String = {
-    val stackTraceString =
-      if (stackTrace == null) "null" else stackTrace.map("        " + _).mkString("\n")
-    s"$className ($description)\n$stackTraceString"
-  }
+  override def toErrorString: String = Utils.exceptionString(className, description, stackTrace)
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/spark/blob/6afca2d1/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
----------------------------------------------------------------------
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 0b2d9db..bb1fcc8 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
@@ -17,6 +17,7 @@
 
 package org.apache.spark.deploy.master
 
+import java.net.URLEncoder
 import java.text.SimpleDateFormat
 import java.util.Date
 
@@ -30,7 +31,6 @@ 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}
@@ -642,10 +642,7 @@ private[spark] class Master(
       waitingApps -= app
 
       // If application events are logged, use them to rebuild the UI
-      if (!rebuildSparkUI(app)) {
-        // Avoid broken links if the UI is not reconstructed
-        app.desc.appUiUrl = ""
-      }
+      rebuildSparkUI(app)
 
       for (exec <- app.executors.values) {
         exec.worker.removeExecutor(exec)
@@ -667,29 +664,47 @@ private[spark] class Master(
    */
   def rebuildSparkUI(app: ApplicationInfo): Boolean = {
     val appName = app.desc.name
-    val eventLogDir = app.desc.eventLogDir.getOrElse { return false }
+    val eventLogDir = app.desc.eventLogDir.getOrElse {
+      // Event logging is not enabled for this application
+      app.desc.appUiUrl = "/history/not-found"
+      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(
-          new SparkConf, replayBus, appName + " (completed)", "/history/" + app.id)
-        replayBus.replay()
-        app.desc.appUiUrl = ui.basePath
-        appIdToUI(app.id) = ui
-        webUi.attachSparkUI(ui)
-        return true
-      } catch {
-        case e: Exception =>
-          logError("Exception in replaying log for application %s (%s)".format(appName, app.id), e)
-      }
-    } else {
-      logWarning("Application %s (%s) has no valid logs: %s".format(appName, app.id, eventLogDir))
+
+    if (eventLogPaths.isEmpty) {
+      // Event logging is enabled for this application, but no event logs are found
+      val title = s"Application history not found (${app.id})"
+      var msg = s"No event logs found for application $appName in $eventLogDir."
+      logWarning(msg)
+      msg += " Did you specify the correct logging directory?"
+      msg = URLEncoder.encode(msg, "UTF-8")
+      app.desc.appUiUrl = s"/history/not-found?msg=$msg&title=$title"
+      return false
+    }
+
+    try {
+      val replayBus = new ReplayListenerBus(eventLogPaths, fileSystem, compressionCodec)
+      val ui = new SparkUI(new SparkConf, replayBus, appName + " (completed)", "/history/" + app.id)
+      replayBus.replay()
+      appIdToUI(app.id) = ui
+      webUi.attachSparkUI(ui)
+      // Application UI is successfully rebuilt, so link the Master UI to it
+      app.desc.appUiUrl = ui.basePath
+      true
+    } catch {
+      case e: Exception =>
+        // Relay exception message to application UI page
+        val title = s"Application history load error (${app.id})"
+        val exception = URLEncoder.encode(Utils.exceptionString(e), "UTF-8")
+        var msg = s"Exception in replaying log for application $appName!"
+        logError(msg, e)
+        msg = URLEncoder.encode(msg, "UTF-8")
+        app.desc.appUiUrl = s"/history/not-found?msg=$msg&exception=$exception&title=$title"
+        false
     }
-    false
   }
 
   /** Generate a new app ID given a app's submission date */

http://git-wip-us.apache.org/repos/asf/spark/blob/6afca2d1/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala
index 34fa142..4588c13 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala
@@ -28,7 +28,7 @@ import org.json4s.JValue
 import org.apache.spark.deploy.{ExecutorState, JsonProtocol}
 import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState}
 import org.apache.spark.deploy.master.ExecutorInfo
-import org.apache.spark.ui.{WebUIPage, UIUtils}
+import org.apache.spark.ui.{UIUtils, WebUIPage}
 import org.apache.spark.util.Utils
 
 private[spark] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app") {

http://git-wip-us.apache.org/repos/asf/spark/blob/6afca2d1/core/src/main/scala/org/apache/spark/deploy/master/ui/HistoryNotFoundPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/HistoryNotFoundPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/HistoryNotFoundPage.scala
new file mode 100644
index 0000000..d8daff3
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/HistoryNotFoundPage.scala
@@ -0,0 +1,73 @@
+/*
+ * 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.master.ui
+
+import java.net.URLDecoder
+import javax.servlet.http.HttpServletRequest
+
+import scala.xml.Node
+
+import org.apache.spark.ui.{UIUtils, WebUIPage}
+
+private[spark] class HistoryNotFoundPage(parent: MasterWebUI)
+  extends WebUIPage("history/not-found") {
+
+  /**
+   * Render a page that conveys failure in loading application history.
+   *
+   * This accepts 3 HTTP parameters:
+   *   msg = message to display to the user
+   *   title = title of the page
+   *   exception = detailed description of the exception in loading application history (if any)
+   *
+   * Parameters "msg" and "exception" are assumed to be UTF-8 encoded.
+   */
+  def render(request: HttpServletRequest): Seq[Node] = {
+    val titleParam = request.getParameter("title")
+    val msgParam = request.getParameter("msg")
+    val exceptionParam = request.getParameter("exception")
+
+    // If no parameters are specified, assume the user did not enable event logging
+    val defaultTitle = "Event logging is not enabled"
+    val defaultContent =
+      <div class="row-fluid">
+        <div class="span12" style="font-size:14px">
+          No event logs were found for this application! To
+          <a href="http://spark.apache.org/docs/latest/monitoring.html">enable event logging</a>,
+          set <span style="font-style:italic">spark.eventLog.enabled</span> to true and
+          <span style="font-style:italic">spark.eventLog.dir</span> to the directory to which your
+          event logs are written.
+        </div>
+      </div>
+
+    val title = Option(titleParam).getOrElse(defaultTitle)
+    val content = Option(msgParam)
+      .map { msg => URLDecoder.decode(msg, "UTF-8") }
+      .map { msg =>
+        <div class="row-fluid">
+          <div class="span12" style="font-size:14px">{msg}</div>
+        </div> ++
+        Option(exceptionParam)
+          .map { e => URLDecoder.decode(e, "UTF-8") }
+          .map { e => <pre>{e}</pre> }
+          .getOrElse(Seq.empty)
+      }.getOrElse(defaultContent)
+
+    UIUtils.basicSparkPage(content, title)
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/6afca2d1/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
----------------------------------------------------------------------
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 a18b39f..16aa049 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
@@ -21,7 +21,7 @@ import org.apache.spark.Logging
 import org.apache.spark.deploy.master.Master
 import org.apache.spark.ui.{SparkUI, WebUI}
 import org.apache.spark.ui.JettyUtils._
-import org.apache.spark.util.{AkkaUtils, Utils}
+import org.apache.spark.util.AkkaUtils
 
 /**
  * Web UI server for the standalone master.
@@ -38,6 +38,7 @@ class MasterWebUI(val master: Master, requestedPort: Int)
   /** Initialize all components of the server. */
   def initialize() {
     attachPage(new ApplicationPage(this))
+    attachPage(new HistoryNotFoundPage(this))
     attachPage(new MasterPage(this))
     attachHandler(createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR, "/static"))
     master.masterMetricsSystem.getServletHandlers.foreach(attachHandler)

http://git-wip-us.apache.org/repos/asf/spark/blob/6afca2d1/core/src/main/scala/org/apache/spark/util/Utils.scala
----------------------------------------------------------------------
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 10c33d6..5784e97 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -44,7 +44,7 @@ import org.apache.spark.executor.ExecutorUncaughtExceptionHandler
 import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance}
 
 /** CallSite represents a place in user code. It can have a short and a long form. */
-private[spark] case class CallSite(val short: String, val long: String)
+private[spark] case class CallSite(short: String, long: String)
 
 /**
  * Various utility methods used by Spark.
@@ -1291,4 +1291,19 @@ private[spark] object Utils extends Logging {
     }
   }
 
+  /** Return a nice string representation of the exception, including the stack trace. */
+  def exceptionString(e: Exception): String = {
+    if (e == null) "" else exceptionString(getFormattedClassName(e), e.getMessage, e.getStackTrace)
+  }
+
+  /** Return a nice string representation of the exception, including the stack trace. */
+  def exceptionString(
+      className: String,
+      description: String,
+      stackTrace: Array[StackTraceElement]): String = {
+    val desc = if (description == null) "" else description
+    val st = if (stackTrace == null) "" else stackTrace.map("        " + _).mkString("\n")
+    s"$className: $desc\n$st"
+  }
+
 }