You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by andrewor14 <gi...@git.apache.org> on 2014/03/22 06:36:45 UTC

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

GitHub user andrewor14 opened a pull request:

    https://github.com/apache/spark/pull/204

    [SPARK-1276] Add a HistoryServer to render persisted UI

    Currently, a persisted UI can only be rendered through the standalone Master. This greatly limits the use case of the new feature of being able to log the details of a Spark application as events, since many people also run Spark on Yarn / Mesos.
    
    This PR introduces a new entity called the HistoryServer, which, given a log directory, keeps track of all completed applications independently of a Spark Master. Unlike Master, the HistoryServer needs not be running while the application is still running. It is relatively light-weight in that it only maintains static information of applications after-the-fact.
    
    To quickly test it out, generate event logs with ```spark.eventLog.enabled=true``` and run ```sbin/start-history-server.sh <log-dir-path>```. Your HistoryServer awaits on port 18080.
    
    A few other changes introduced in this PR include refactoring the WebUI interface, which is beginning to have a lot of duplicate code now that we add more functionality to it. Two new SparkListenerEvents have been introduced (SparkListenerApplicationStart/End) to keep track of application name and start/finish times. This PR also clarifies the semantics of the ReplayListenerBus introduced in #42.
    
    A potential TODO in the future (not part of this PR) is to render live event logging applications in addition to just completed applications. This is useful if an application fails, in which case our current HistoryServer does not render the associated UI unless the user manually signals application completion. Processing the event logs in this case becomes significantly more complicated, however, because we must deal with multiple levels of streams that may each have arbitrary behavior if we want to avoid processing the entire file over and over again.
    
    Comments and feedback are most welcome.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/andrewor14/spark master

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/204.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #204
    
----
commit c086bd5c6837a98d3c989c43f2b75aeaa0e5eff0
Author: Andrew Or <an...@gmail.com>
Date:   2014-03-20T19:43:16Z

    Add HistoryServer and scripts ++ Refactor WebUI interface
    
    HistoryServer can be launched with ./sbin/start-history-server.sh <log-dir>
    and stopped with ./sbin/stop-history-server.sh. This commit also involves
    refactoring all the UIs to avoid duplicate code.

commit 8aac16355329809b11c76430fa8737d328f2e962
Author: Andrew Or <an...@gmail.com>
Date:   2014-03-20T21:34:34Z

    Add basic application table

commit 758441890dc86c8ed069e6c684b21528038f2ff7
Author: Andrew Or <an...@gmail.com>
Date:   2014-03-21T04:59:34Z

    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.

commit 60bc6d57577742e861d62c183ec56d9893e3ea6a
Author: Andrew Or <an...@gmail.com>
Date:   2014-03-22T01:17:43Z

    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.

commit 5dbfbb47826ea2edbf8cf2100228bddb5be473f8
Author: Andrew Or <an...@gmail.com>
Date:   2014-03-22T01:54:28Z

    Merge branch 'master' of github.com:apache/spark
    
    Conflicts:
    	core/src/main/scala/org/apache/spark/deploy/DeployWebUI.scala
    	core/src/main/scala/org/apache/spark/deploy/WebUI.scala
    	core/src/main/scala/org/apache/spark/deploy/master/Master.scala
    	core/src/main/scala/org/apache/spark/ui/WebUI.scala

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11456122
  
    --- Diff: core/src/main/scala/org/apache/spark/ui/WebUI.scala ---
    @@ -20,6 +20,25 @@ package org.apache.spark.ui
     import java.text.SimpleDateFormat
     import java.util.Date
     
    +private[spark] abstract class WebUI(name: String) {
    --- End diff --
    
    Is this abstraction actually used anywhere except for being extended? I couldn't find any cases where this was used.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40045811
  
     Merged build triggered. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39997905
  
    Hm, I was able to force-touch an APPLICATION_COMPLETE file and have the history server render it. The "Skipping incomplete application" warning is printed before I manually added the file, which makes sense because it was still running. After I added the file, I no longer get the warning.
    
    As for exposing the configurations, I need to talk to @pwendell about it. Right now SPARK_DAEMON_* variables are documented under the standalone page, but the server is designed for the YARN / Mesos mode. But you're right, all this information should be documented somewhere.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11459038
  
    --- Diff: 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://<driver-node>: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 <base-logging-directory>
    +
    +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://<server-url>: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:
    +
    +<table class="table">
    +  <tr><th style="width:21%">Environment Variable</th><th>Meaning</th></tr>
    +  <tr>
    +    <td><code>SPARK_DAEMON_MEMORY</code></td>
    +    <td>Memory to allocate to the history server. (default: 512m).</td>
    +  </tr>
    +  <tr>
    +    <td><code>SPARK_DAEMON_JAVA_OPTS</code></td>
    +    <td>JVM options for the history server (default: none).</td>
    +  </tr>
    +</table>
    +
    +Further, the history server can be configured as follows:
    +
    +<table class="table">
    +  <tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
    +  <tr>
    +    <td>spark.history.updateInterval</td>
    +    <td>10</td>
    +    <td>
    +      The period at which information displayed by this history server is updated. Each update
    --- End diff --
    
    I'd say "The period, in seconds, at which..."


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40052033
  
    Merged build finished. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40039312
  
    Merged build started. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40059950
  
    Jenkins, test this please.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11459091
  
    --- Diff: 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://<driver-node>: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 <base-logging-directory>
    +
    +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://<server-url>: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:
    +
    +<table class="table">
    +  <tr><th style="width:21%">Environment Variable</th><th>Meaning</th></tr>
    +  <tr>
    +    <td><code>SPARK_DAEMON_MEMORY</code></td>
    +    <td>Memory to allocate to the history server. (default: 512m).</td>
    +  </tr>
    +  <tr>
    +    <td><code>SPARK_DAEMON_JAVA_OPTS</code></td>
    +    <td>JVM options for the history server (default: none).</td>
    +  </tr>
    +</table>
    +
    +Further, the history server can be configured as follows:
    +
    +<table class="table">
    +  <tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
    +  <tr>
    +    <td>spark.history.updateInterval</td>
    +    <td>10</td>
    +    <td>
    +      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.
    +    </td>
    +  </tr>
    +  <tr>
    +    <td>spark.history.retainedApplications</td>
    +    <td>250</td>
    +    <td>
    +      The number of application UIs to retain. If this cap is exceeded, then the least recently
    +      updated applications will be removed.
    --- End diff --
    
    Maybe just say the "oldest applications"? Fine either way though, maybe "least recently updated" is more precise.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11426483
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -0,0 +1,265 @@
    +/*
    + * 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.collection.mutable
    +import scala.concurrent._
    +import scala.concurrent.ExecutionContext.Implicits.global
    +
    +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.{ApplicationEventListener, ReplayListenerBus}
    +
    +/**
    + * 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. 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)
    +  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 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 val handlers = Seq[ServletContextHandler](
    +    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]()
    --- End diff --
    
    For sure. We can simply maintain a simple timestamp threshold, where all applications that are last updated before this threshold will not be rendered. (See the latest commit for more details)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11456690
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -0,0 +1,322 @@
    +/*
    + * 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.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.{ApplicationEventListener, ReplayListenerBus}
    +
    +/**
    + * 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. 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)
    +  extends SparkUIContainer("History Server") with Logging {
    +
    +  import 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 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 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"),
    +    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]()
    +
    +  /**
    +   * 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. */
    +  override def bind() {
    +    try {
    +      serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, conf))
    +      logInfo("Started HistoryServer at http://%s:%d".format(publicHost, boundPort))
    +    } catch {
    +      case e: Exception =>
    +        logError("Failed to bind HistoryServer", e)
    +        System.exit(1)
    +    }
    +    checkForLogs()
    +  }
    +
    +  /**
    +   * 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
    +   * information for this application.
    +   *
    +   * If the logs for an existing finished application are no longer found, the server
    +   * removes all associated information and detaches the SparkUI.
    +   */
    +  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 mostRecentAppIds = logDirs.map { dir => getAppId(dir.getPath.toString) }
    +        appIdToInfo.foreach { case (appId, info) =>
    +          if (!mostRecentAppIds.contains(appId)) {
    +            detachUI(info.ui)
    +            appIdToInfo.remove(appId)
    +            updateTimeThreshold = -1L
    --- End diff --
    
    Why does this need to be reset here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11317150
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -0,0 +1,221 @@
    +/*
    + * 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 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
    +
    +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.{ApplicationEventListener, ReplayListenerBus}
    +
    +/**
    + * 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. 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)
    +  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())
    --- End diff --
    
    Yes, this was an old change that preceded the public dns patch.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40046213
  
    Not from my side (unless the tests fail...)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40044103
  
    Merged build started. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11256081
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -0,0 +1,221 @@
    +/*
    + * 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 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
    +
    +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.{ApplicationEventListener, ReplayListenerBus}
    +
    +/**
    + * 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. 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)
    +  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 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
    +
    +  private val handlers = Seq[ServletContextHandler](
    +    createStaticHandler(HistoryServer.STATIC_RESOURCE_DIR, "/static"),
    +    createServletHandler("/",
    +      (request: HttpServletRequest) => indexPage.render(request), securityMgr = securityManager)
    +  )
    +
    +  val appIdToInfo = mutable.HashMap[String, ApplicationHistoryInfo]()
    +
    +  /** 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 bind HistoryServer", e)
    +        System.exit(1)
    +    }
    +    checkForLogs()
    +  }
    +
    +  /**
    +   * 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
    +   * information for this application.
    +   *
    +   * 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 {
    +        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)
    +          }
    +        }
    +      }
    +      asyncCheck.onFailure { case t =>
    +        logError("Unable to synchronize HistoryServer with files on disk: ", t)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * 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(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 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) {
    +        attachUI(ui)
    +        val appName = if (appListener.applicationStarted) appListener.appName else appId
    +        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("Skipping incomplete application: %s".format(logPath))
    +    }
    +    replayBus.stop()
    --- End diff --
    
    This is probably what is causing the exception with:
    
    java.io.IOException: Filesystem closed
            at org.apache.hadoop.hdfs.DFSClient.checkOpen(DFSClient.java:398)
    
    In stop() it is closing the filesystem. That filesystem is cached and used throughout the process so you can't close it. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11388641
  
    --- Diff: 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.
    --- End diff --
    
    It would be nice to have a small section of documentation about start/stopping the history server as well as this setting on configuring the memory.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11459562
  
    --- Diff: core/src/main/scala/org/apache/spark/ui/WebUI.scala ---
    @@ -20,6 +20,25 @@ package org.apache.spark.ui
     import java.text.SimpleDateFormat
     import java.util.Date
     
    +private[spark] abstract class WebUI(name: String) {
    --- End diff --
    
    The main reason for doing this is to make a common interface for MasterWebUI and HistoryServer to attach SparkUI's without each having to re-implement the logic themselves.
    
    In the future, when we want to extend SparkUI to include other tabs (e.g. Streaming), this will be more necessary. An example of that is in this [sub-PR](https://github.com/tdas/spark/pull/2).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11469154
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -0,0 +1,287 @@
    +/*
    + * 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
    --- End diff --
    
    My bad - was just an IDE error on my side


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11471050
  
    --- Diff: core/src/main/scala/org/apache/spark/ui/WebUI.scala ---
    @@ -20,6 +20,25 @@ package org.apache.spark.ui
     import java.text.SimpleDateFormat
     import java.util.Date
     
    +private[spark] abstract class WebUI(name: String) {
    --- End diff --
    
    Okay - sounds good.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-38881861
  
    Can one of the admins verify this patch?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11469937
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala ---
    @@ -0,0 +1,51 @@
    +/*
    + * 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
    --- End diff --
    
    I found this a little hard to grok. The meat of this seems to be that basically this assumes it only sees events from one application. What about something like:
    ```
    This listener expects to hear events from a single application only. If events from multiple
    applications are seen, the behavior is unspecified.
    ```
    
    Then even if this implementation evolves to actually track stuff using the other events, you've already made a general carve-out in the javadoc with the high level assumption.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11469740
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -90,9 +97,115 @@ 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)
    +
    +  /**
    +   * Stop logging events.
    +   * In addition, create an empty special file to indicate application completion.
    +   */
    +  def stop() = {
    +    logger.newFile(APPLICATION_COMPLETE)
    +    logger.stop()
    +  }
    +}
    +
    +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 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)
    +  }
     
    -  def stop() = logger.stop()
    +  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, "")
    +    } 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("<Unknown>"),
    +        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)
    --- End diff --
    
    I guess the assumption here is that creating the codec instance is cheap? Just mentioning because how it is now, we'll create codec objects way more than we ever use them. In the rest of spark we only create codecs when we use them, it might be nice to just create them downstream.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40036585
  
     Merged build triggered. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11456639
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -0,0 +1,322 @@
    +/*
    + * 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.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.{ApplicationEventListener, ReplayListenerBus}
    +
    +/**
    + * 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. 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)
    +  extends SparkUIContainer("History Server") with Logging {
    +
    +  import 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 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 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"),
    +    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]()
    +
    +  /**
    +   * 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. */
    +  override def bind() {
    +    try {
    +      serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, conf))
    +      logInfo("Started HistoryServer at http://%s:%d".format(publicHost, boundPort))
    +    } catch {
    +      case e: Exception =>
    +        logError("Failed to bind HistoryServer", e)
    +        System.exit(1)
    +    }
    +    checkForLogs()
    +  }
    +
    +  /**
    +   * 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
    +   * information for this application.
    +   *
    +   * If the logs for an existing finished application are no longer found, the server
    +   * removes all associated information and detaches the SparkUI.
    +   */
    +  def checkForLogs() = synchronized {
    --- End diff --
    
    The IndexPage might have a button in the future that calls this. Right now there's no mechanism for manually refreshing, but I intend to add that in a future PR.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40062917
  
    Merged build finished. All automated tests passed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39155060
  
     Build triggered. Build is starting -or- tests failed to complete.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40052187
  
     Merged build triggered. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40046214
  
     Merged build triggered. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39453542
  
    Also to clarify that the history files are actually first written to the users staging directory, which is only user readable/writable, and then moved over to done_intermediate when the job finishes/fails.  MR does generally make sure the permissions on directories/files it creating are set appropriately or according to configs the admin specifies.  
    
    Note that I'm definitely not saying we have to do it like MR, just giving an example of how it does it.    As long as things are protected its fine.  And ideally have at least some basic management system to remove old ones.  We can add the management stuff later, although would be nice for 1.0. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40038198
  
    Merged build finished. All automated tests passed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40052200
  
    Merged build started. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11456364
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -0,0 +1,322 @@
    +/*
    + * 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.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.{ApplicationEventListener, ReplayListenerBus}
    +
    +/**
    + * 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. 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)
    +  extends SparkUIContainer("History Server") with Logging {
    +
    +  import 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 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 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"),
    +    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]()
    +
    +  /**
    +   * 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. */
    +  override def bind() {
    +    try {
    +      serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, conf))
    +      logInfo("Started HistoryServer at http://%s:%d".format(publicHost, boundPort))
    +    } catch {
    +      case e: Exception =>
    +        logError("Failed to bind HistoryServer", e)
    +        System.exit(1)
    +    }
    +    checkForLogs()
    +  }
    +
    +  /**
    +   * 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
    +   * information for this application.
    +   *
    +   * If the logs for an existing finished application are no longer found, the server
    +   * removes all associated information and detaches the SparkUI.
    +   */
    +  def checkForLogs() = synchronized {
    --- End diff --
    
    should this be private?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11469876
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -0,0 +1,287 @@
    +/*
    + * 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.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.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.ui.JettyUtils._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * 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. 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
    + */
    +class HistoryServer(
    +    val baseLogDir: String,
    +    conf: SparkConf)
    +  extends SparkUIContainer("History Server") with Logging {
    +
    +  import 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 = WEB_UI_PORT
    +  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 lastLogCheckTime = -1L
    +
    +  // Number of complete applications found in this directory
    +  private var numApplicationsTotal = 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.
    +   *
    +   * TODO: Add a mechanism to update manually.
    +   */
    +  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"),
    +    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]()
    +
    +  /**
    +   * 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. */
    +  override def bind() {
    +    try {
    +      serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, conf))
    +      logInfo("Started HistoryServer at http://%s:%d".format(publicHost, boundPort))
    +    } catch {
    +      case e: Exception =>
    +        logError("Failed to bind HistoryServer", e)
    +        System.exit(1)
    +    }
    +  }
    +
    +  /**
    +   * 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
    +   * information for this application.
    +   *
    +   * If the logs for an existing finished application are no longer found, the server
    +   * removes all associated information and detaches the SparkUI.
    +   */
    +  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]()
    +        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 }
    +
    +        // Remove any applications that should no longer be retained
    +        appIdToInfo.foreach { case (appId, info) =>
    +          if (!retainedAppIds.contains(appId)) {
    +            detachUI(info.ui)
    +            appIdToInfo.remove(appId)
    +          }
    +        }
    +
    +        // 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)) {
    +            renderSparkUI(dir, info)
    +          }
    +        }
    +
    +        // 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)
    +      }
    +    } else {
    +      logWarning("Attempted to check for event log updates before binding the server.")
    +    }
    +  }
    +
    +  /**
    +   * 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 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
    --- End diff --
    
    Is there any reason the `SparkUI` couldn't be modify to just collect and expose the fields that the `ApplicaitonEventListener` currently deals with? Then you wouldn't have to create this extra listener class. Doesn't the SparkUI class need to know all of this stuff anyways?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40051269
  
    
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/13989/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40115878
  
    Thanks @pwendell and @tgraves! :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11456895
  
    --- Diff: core/src/main/scala/org/apache/spark/SparkContext.scala ---
    @@ -1115,6 +1114,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, sparkUser))
    +  }
    +
    +  /**
    +   * 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() {
    --- End diff --
    
    This is different from the Shutdown event. The StageCompletion event will still be processed in the regular listener bus thread. It is true that the ApplicationEnd event may be processed before the StageCompletion event, however. A proper way of dealing with this is in #366, where the ordering is preserved but all events are guaranteed to be processed to completion.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11459953
  
    --- Diff: 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 <base-log-dir> [<web-ui-port>]
    +#   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 <base-log-dir> [<web-ui-port>]"
    +  echo "Example: ./start-history-server.sh /tmp/spark-events 18080"
    --- End diff --
    
    Let's bump the port to a config option as per our discussion.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11317221
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -0,0 +1,221 @@
    +/*
    + * 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 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
    +
    +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.{ApplicationEventListener, ReplayListenerBus}
    +
    +/**
    + * 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. 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)
    +  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 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
    +
    +  private val handlers = Seq[ServletContextHandler](
    +    createStaticHandler(HistoryServer.STATIC_RESOURCE_DIR, "/static"),
    +    createServletHandler("/",
    +      (request: HttpServletRequest) => indexPage.render(request), securityMgr = securityManager)
    +  )
    +
    +  val appIdToInfo = mutable.HashMap[String, ApplicationHistoryInfo]()
    +
    +  /** 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 bind HistoryServer", e)
    +        System.exit(1)
    +    }
    +    checkForLogs()
    +  }
    +
    +  /**
    +   * 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
    +   * information for this application.
    +   *
    +   * 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 {
    +        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)
    +          }
    +        }
    +      }
    +      asyncCheck.onFailure { case t =>
    +        logError("Unable to synchronize HistoryServer with files on disk: ", t)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * 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(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 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) {
    +        attachUI(ui)
    +        val appName = if (appListener.applicationStarted) appListener.appName else appId
    +        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("Skipping incomplete application: %s".format(logPath))
    +    }
    +    replayBus.stop()
    --- End diff --
    
    Sure, we can close it only when we terminate the HistoryServer. Great find.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11256259
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -0,0 +1,221 @@
    +/*
    + * 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 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
    +
    +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.{ApplicationEventListener, ReplayListenerBus}
    +
    +/**
    + * 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. 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)
    +  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 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
    +
    +  private val handlers = Seq[ServletContextHandler](
    +    createStaticHandler(HistoryServer.STATIC_RESOURCE_DIR, "/static"),
    +    createServletHandler("/",
    +      (request: HttpServletRequest) => indexPage.render(request), securityMgr = securityManager)
    +  )
    +
    +  val appIdToInfo = mutable.HashMap[String, ApplicationHistoryInfo]()
    +
    +  /** 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 bind HistoryServer", e)
    +        System.exit(1)
    +    }
    +    checkForLogs()
    +  }
    +
    +  /**
    +   * 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
    +   * information for this application.
    +   *
    +   * 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 {
    +        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)
    +          }
    +        }
    +      }
    +      asyncCheck.onFailure { case t =>
    +        logError("Unable to synchronize HistoryServer with files on disk: ", t)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * 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(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 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) {
    +        attachUI(ui)
    +        val appName = if (appListener.applicationStarted) appListener.appName else appId
    +        ui.setAppName("%s (finished)".format(appName))
    +        val startTime = appListener.startTime
    +        val endTime = appListener.endTime
    +        val info = ApplicationHistoryInfo(appName, startTime, endTime, lastUpdated, logPath, ui)
    --- End diff --
    
    How big is this object?  It might be good to add a maximum number to keep in memory.  Having a way to get at the ones not in memory would be good though too. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39919444
  
    @tgravescs Your browser is not funky. The current implementation triggers an asynchronous log fetch every time the browser refreshes, so by the time the first refresh is done, the logs have not been processed yet. I realize this behavior is probably not desired and I will push out a fix shortly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11458956
  
    --- Diff: core/src/main/scala/org/apache/spark/ui/WebUI.scala ---
    @@ -20,6 +20,25 @@ package org.apache.spark.ui
     import java.text.SimpleDateFormat
     import java.util.Date
     
    +private[spark] abstract class WebUI(name: String) {
    --- End diff --
    
    Just to enforce conventions amongst the UI's?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40053882
  
    Jenkins, test this please.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39156083
  
    Build is starting -or- tests failed to complete.
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/13613/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39155141
  
    Build finished. Build is starting -or- tests failed to complete.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40050718
  
    Merged build finished. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-38501783
  
     Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39699926
  
     Merged build triggered. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11459453
  
    --- Diff: core/src/main/scala/org/apache/spark/ui/WebUI.scala ---
    @@ -20,6 +20,25 @@ package org.apache.spark.ui
     import java.text.SimpleDateFormat
     import java.util.Date
     
    +private[spark] abstract class WebUI(name: String) {
    --- End diff --
    
    It abstracts duplicate code away. Right now each of SparkUI, MasterWebUI, WorkerWebUI and HistoryServer define their own `serverInfo`, `stop()` and `boundPort` etc. which is common among all these classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11459765
  
    --- Diff: 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://<driver-node>: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 <base-logging-directory>
    +
    +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://<server-url>: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:
    +
    +<table class="table">
    +  <tr><th style="width:21%">Environment Variable</th><th>Meaning</th></tr>
    +  <tr>
    +    <td><code>SPARK_DAEMON_MEMORY</code></td>
    +    <td>Memory to allocate to the history server. (default: 512m).</td>
    +  </tr>
    +  <tr>
    +    <td><code>SPARK_DAEMON_JAVA_OPTS</code></td>
    +    <td>JVM options for the history server (default: none).</td>
    +  </tr>
    +</table>
    +
    +Further, the history server can be configured as follows:
    +
    +<table class="table">
    +  <tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
    +  <tr>
    +    <td>spark.history.updateInterval</td>
    +    <td>10</td>
    +    <td>
    +      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.
    +    </td>
    +  </tr>
    +  <tr>
    +    <td>spark.history.retainedApplications</td>
    +    <td>250</td>
    +    <td>
    +      The number of application UIs to retain. If this cap is exceeded, then the least recently
    +      updated applications will be removed.
    --- End diff --
    
    Hm I didn't realize the formatting is screwed up though


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39940453
  
     Merged build triggered. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39940524
  
    Build finished. All automated tests passed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40036592
  
    Merged build started. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40050781
  
    Tests pass locally for me


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11456820
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -0,0 +1,322 @@
    +/*
    + * 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.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.{ApplicationEventListener, ReplayListenerBus}
    +
    +/**
    + * 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. 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)
    +  extends SparkUIContainer("History Server") with Logging {
    +
    +  import 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 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 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"),
    +    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]()
    +
    +  /**
    +   * 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. */
    +  override def bind() {
    +    try {
    +      serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, conf))
    +      logInfo("Started HistoryServer at http://%s:%d".format(publicHost, boundPort))
    +    } catch {
    +      case e: Exception =>
    +        logError("Failed to bind HistoryServer", e)
    +        System.exit(1)
    +    }
    +    checkForLogs()
    +  }
    +
    +  /**
    +   * Check for any updates to event logs in the base directory. This is only effective once
    --- End diff --
    
    This function is pretty complicated - would it make more sense to decouple a few of the pieces into different functions? I think there are a few things that are sort of independent.
    
    1) Garbage collecting applications whose underlying logs have been deleted. In this case by the way, would it be simpler to just not do this? Why not just let these fall out of scope naturally when the limit is hit...
    2) Detecting new applications which were not previously loaded, but should be.
    3) Loading applications which we've decided should be loaded.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40050675
  
    Jenkins, retest this please.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11254082
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -0,0 +1,221 @@
    +/*
    + * 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 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
    +
    +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.{ApplicationEventListener, ReplayListenerBus}
    +
    +/**
    + * 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. 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)
    +  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 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
    +
    +  private val handlers = Seq[ServletContextHandler](
    +    createStaticHandler(HistoryServer.STATIC_RESOURCE_DIR, "/static"),
    +    createServletHandler("/",
    +      (request: HttpServletRequest) => indexPage.render(request), securityMgr = securityManager)
    +  )
    +
    +  val appIdToInfo = mutable.HashMap[String, ApplicationHistoryInfo]()
    +
    +  /** 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 bind HistoryServer", e)
    +        System.exit(1)
    +    }
    +    checkForLogs()
    +  }
    +
    +  /**
    +   * 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
    +   * information for this application.
    +   *
    +   * 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 {
    +        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)
    +          }
    +        }
    +      }
    +      asyncCheck.onFailure { case t =>
    +        logError("Unable to synchronize HistoryServer with files on disk: ", t)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * 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(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 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) {
    +        attachUI(ui)
    +        val appName = if (appListener.applicationStarted) appListener.appName else appId
    +        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("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 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
    +    }
    +  }
    +
    +  /** Return whether the last log check has happened sufficiently long ago. */
    +  private def logCheckReady: Boolean = {
    +    System.currentTimeMillis - lastLogCheck > HistoryServer.UPDATE_INTERVAL_SECONDS * 1000
    --- End diff --
    
    might be nice to make this configurable eventually.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11470003
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -90,9 +97,115 @@ 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)
    +
    +  /**
    +   * Stop logging events.
    +   * In addition, create an empty special file to indicate application completion.
    +   */
    +  def stop() = {
    +    logger.newFile(APPLICATION_COMPLETE)
    +    logger.stop()
    +  }
    +}
    +
    +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 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)
    +  }
     
    -  def stop() = logger.stop()
    +  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, "")
    +    } 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("<Unknown>"),
    +        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)
    --- End diff --
    
    Also the indentation is weird. Don't know why I didn't catch that


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40011833
  
    Merged build started. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-38344040
  
    Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40052042
  
    Jenkins, test this please.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11469880
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -90,9 +97,115 @@ 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)
    +
    +  /**
    +   * Stop logging events.
    +   * In addition, create an empty special file to indicate application completion.
    +   */
    +  def stop() = {
    +    logger.newFile(APPLICATION_COMPLETE)
    +    logger.stop()
    +  }
    +}
    +
    +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"
    --- End diff --
    
    Currently we don't handle the failure case, however. I think when an application fails there are no guarantees that it may even write the file, so I'm not sure if it makes sense to have an APPLICATION_FAILED file (unlike tasks, applications don't have their own drivers!)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39206395
  
    I haven't had a chance to look at this in detail yet, but I'm wondering if this handles security at all?
    I'm assuming the web UI still has the filters and acls I added and just uses the same to authorize a user to view a application? Does the history server run as a super user?
    
    In the description you say the history server reads from a log directory.  How are the files in that directory protected from other people reading them?    Can it read from multiple log directories?
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39152812
  
    Build finished. Build is starting -or- tests failed to complete.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11389162
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -0,0 +1,265 @@
    +/*
    + * 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.collection.mutable
    +import scala.concurrent._
    +import scala.concurrent.ExecutionContext.Implicits.global
    +
    +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.{ApplicationEventListener, ReplayListenerBus}
    +
    +/**
    + * 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. 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)
    +  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 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 val handlers = Seq[ServletContextHandler](
    +    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 {
    +      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)
    +        System.exit(1)
    +    }
    +    checkForLogs()
    +  }
    +
    +  /**
    +   * 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
    +   * information for this application.
    +   *
    +   * 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 {
    +        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) && !appIdBlacklist.contains(appId)) {
    +            maybeRenderUI(appId, path, lastUpdated)
    +          }
    +          // If the cap is reached, remove the least recently updated application
    +          if (appIdToInfo.size > RETAINED_APPLICATIONS) {
    +            removeOldestApp()
    +          }
    +        }
    +      }
    +      asyncCheck.onFailure { case t =>
    +        logError("Unable to synchronize HistoryServer with files on disk: ", t)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * 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(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
    +        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))
    +    }
    +  }
    +
    +  /** 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: 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 = {
    +    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)
    +    }
    +  }
    +
    +  /** 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
    + * 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 {
    +  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)
    +
    +  // How many applications to retain
    +  private val RETAINED_APPLICATIONS = conf.getInt("spark.deploy.retainedApplications", 20)
    --- End diff --
    
    hopefully the default can be a bit higher.  I would like to see it in the thousands, but I guess the default memory is only 512.    Do you know how big ApplicationHistoryInfo gets?  


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11469656
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -90,9 +97,115 @@ 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)
    +
    +  /**
    +   * Stop logging events.
    +   * In addition, create an empty special file to indicate application completion.
    +   */
    +  def stop() = {
    +    logger.newFile(APPLICATION_COMPLETE)
    +    logger.stop()
    +  }
    +}
    +
    +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"
    --- End diff --
    
    Would it make sense to have two cases here? APPLICATION_COMPLETE_SUCCESS and APPLICATION_COMPLETE_FAILURE? or maybe APPLICATION_COMPLETE  and APPLICATION_FAILURE? The former would be more consistent with some other parts of Spark where "complete" does not necessarily mean success or failure, just that it's finished.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39940616
  
    This looks good, I want to do a full pass tomorrow morning but in general I think this is in good shape.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11455914
  
    --- Diff: core/src/main/scala/org/apache/spark/SparkContext.scala ---
    @@ -1115,6 +1114,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, sparkUser))
    +  }
    +
    +  /**
    +   * 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() {
    --- End diff --
    
    For example say the user code does this:
    
    val x = rdd.count()
    sc.stop()
    
    Couldn't there be a stage completion event that is in the queue and gets skipped here?



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40050925
  
    Merged build started. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39790164
  
    I'm getting an exception trying to run on a secure hadoop cluster with the logging enabled.  It looks like it probably isn't propogating tokens properly.   I'll try to dig into it more later.  Note that this is without the history server running, just the ui logging.
    
    
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-38345707
  
    All automated tests passed.
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/13337/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39155380
  
     Merged build triggered. Build is starting -or- tests failed to complete.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-38507698
  
    All automated tests passed.
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/13406/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40015884
  
    Merged build finished. All automated tests passed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40048196
  
    
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/13983/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11469215
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -0,0 +1,287 @@
    +/*
    + * 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.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.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.ui.JettyUtils._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * 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. 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
    + */
    +class HistoryServer(
    +    val baseLogDir: String,
    +    conf: SparkConf)
    +  extends SparkUIContainer("History Server") with Logging {
    +
    +  import 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 = WEB_UI_PORT
    +  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 lastLogCheckTime = -1L
    +
    +  // Number of complete applications found in this directory
    +  private var numApplicationsTotal = 0
    --- End diff --
    
    could this be more specifically named `numCompletedApplications`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11469955
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -0,0 +1,287 @@
    +/*
    + * 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.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.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.ui.JettyUtils._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * 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. 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
    + */
    +class HistoryServer(
    +    val baseLogDir: String,
    +    conf: SparkConf)
    +  extends SparkUIContainer("History Server") with Logging {
    +
    +  import 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 = WEB_UI_PORT
    +  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 lastLogCheckTime = -1L
    +
    +  // Number of complete applications found in this directory
    +  private var numApplicationsTotal = 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.
    +   *
    +   * TODO: Add a mechanism to update manually.
    +   */
    +  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"),
    +    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]()
    +
    +  /**
    +   * 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. */
    +  override def bind() {
    +    try {
    +      serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, conf))
    +      logInfo("Started HistoryServer at http://%s:%d".format(publicHost, boundPort))
    +    } catch {
    +      case e: Exception =>
    +        logError("Failed to bind HistoryServer", e)
    +        System.exit(1)
    +    }
    +  }
    +
    +  /**
    +   * 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
    +   * information for this application.
    +   *
    +   * If the logs for an existing finished application are no longer found, the server
    +   * removes all associated information and detaches the SparkUI.
    +   */
    +  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]()
    +        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 }
    +
    +        // Remove any applications that should no longer be retained
    +        appIdToInfo.foreach { case (appId, info) =>
    +          if (!retainedAppIds.contains(appId)) {
    +            detachUI(info.ui)
    +            appIdToInfo.remove(appId)
    +          }
    +        }
    +
    +        // 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)) {
    +            renderSparkUI(dir, info)
    +          }
    +        }
    +
    +        // 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)
    +      }
    +    } else {
    +      logWarning("Attempted to check for event log updates before binding the server.")
    +    }
    +  }
    +
    +  /**
    +   * 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 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
    --- End diff --
    
    Even if we do it inside SparkUI, we still need a custom listener to listen for the ApplicationStart events. This unnecessarily complicates the SparkUI, which can just get the appName, for example, from its parent (SparkContext / Master / HistoryServer).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39155142
  
    Build is starting -or- tests failed to complete.
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/13612/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11469853
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -0,0 +1,287 @@
    +/*
    + * 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.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.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.ui.JettyUtils._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * A web server that renders SparkUIs of finished applications.
    --- End diff --
    
    I've been inconsistent. I use "completed" in some places (e.g. EventLoggingInfo) and "finished" in others (here). It's nice to be more consistent.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39152681
  
     Build triggered. Build is starting -or- tests failed to complete.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40052034
  
    
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/13991/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-38501784
  
    Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40046220
  
    Merged build started. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11459249
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -0,0 +1,322 @@
    +/*
    + * 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.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.{ApplicationEventListener, ReplayListenerBus}
    +
    +/**
    + * 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. 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)
    +  extends SparkUIContainer("History Server") with Logging {
    +
    +  import 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 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 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"),
    +    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]()
    +
    +  /**
    +   * 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. */
    +  override def bind() {
    +    try {
    +      serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, conf))
    +      logInfo("Started HistoryServer at http://%s:%d".format(publicHost, boundPort))
    +    } catch {
    +      case e: Exception =>
    +        logError("Failed to bind HistoryServer", e)
    +        System.exit(1)
    +    }
    +    checkForLogs()
    +  }
    +
    +  /**
    +   * Check for any updates to event logs in the base directory. This is only effective once
    --- End diff --
    
    We discussed offline about how this could be re-factored a bit.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40048195
  
    Merged build finished. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40060066
  
     Merged build triggered. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39699932
  
    Merged build started. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11456909
  
    --- Diff: core/src/main/scala/org/apache/spark/ui/WebUI.scala ---
    @@ -20,6 +20,25 @@ package org.apache.spark.ui
     import java.text.SimpleDateFormat
     import java.util.Date
     
    +private[spark] abstract class WebUI(name: String) {
    --- End diff --
    
    Yes, it is an abstract class and cannot be used other than being extended.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11254543
  
    --- Diff: 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 <base-log-dir> [<web-ui-port>]
    +#   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 <base-log-dir> [<web-ui-port>]"
    +  echo "Example: ./start-history-server.sh /tmp/spark-events 18080"
    --- End diff --
    
    What is the recommended way for setting the heap size of the history server?  I see spark-class now has a big case statement using different env variables for different daemon.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39236711
  
    Hey @tgravescs this will pick upon ACL's and filters if configured similar to the other UI's. In terms of HDFS permission, this patch takes a simple approach of having a shared parent directory for all jobs and then each job creates a child directory where it writes its own logs. The user that runs the UI server needs to have read access to all of the child directories. Each job has its own directory though so they can be isolated from eachother.
    
    One thing is, we might modify the way the job logs are written so that when the directory is created we automatically lock it down with certain permissions (this wouldn't affect the code in this patch, only the code that creates the log in the first place). Not sure what the best way to do that is or if there is a comparison here in YARN land.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11456312
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -0,0 +1,322 @@
    +/*
    + * 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.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.{ApplicationEventListener, ReplayListenerBus}
    +
    +/**
    + * 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. 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)
    +  extends SparkUIContainer("History Server") with Logging {
    +
    +  import 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 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 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
    --- End diff --
    
    Is there a code path here where the user can check for logs?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40044025
  
    Jenkins, retest this please


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11458781
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -0,0 +1,322 @@
    +/*
    + * 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.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.{ApplicationEventListener, ReplayListenerBus}
    +
    +/**
    + * 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. 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)
    +  extends SparkUIContainer("History Server") with Logging {
    +
    +  import 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 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 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
    --- End diff --
    
    Not as of now (there will be in the future)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11395293
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala ---
    @@ -98,7 +121,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(_.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)
    --- End diff --
    
    We should probably catch exceptions here also and just skip that directory.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39155385
  
    Merged build started. Build is starting -or- tests failed to complete.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11469140
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -0,0 +1,287 @@
    +/*
    + * 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
    --- End diff --
    
    This and `ServletContextHandler` are unused now


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11426446
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala ---
    @@ -0,0 +1,76 @@
    +/*
    + * 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.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 =
    +      <div class="row-fluid">
    +        <div class="span12">
    +          <ul class="unstyled">
    +            <li><strong>Event Log Location: </strong> {parent.baseLogDir}</li>
    +            <h4>
    +              Showing {parent.appIdToInfo.size}/{parent.getTotalApplications} Finished Applications
    +            </h4>
    +            {appTable}
    +          </ul>
    +        </div>
    +      </div>
    +
    +    UIUtils.basicSparkPage(content, "History Server")
    +  }
    +
    +  private val appHeader = Seq(
    +    "App Name",
    +    "Started",
    +    "Finished",
    +    "Duration",
    +    "Log Directory",
    +    "Last Updated")
    +
    +  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) 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)
    --- End diff --
    
    This is added in the latest commit


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11471015
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -0,0 +1,287 @@
    +/*
    + * 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
    --- End diff --
    
    I figured maybe I just needed to run `sbt/sbt gen-idea` again to get the new dependencies.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39152813
  
    Build is starting -or- tests failed to complete.
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/13610/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11469526
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala ---
    @@ -0,0 +1,72 @@
    +/*
    + * 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.hadoop.fs.Path
    +
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Command-line parser for the master.
    + */
    +private[spark] class HistoryServerArguments(args: Array[String]) {
    +  var logDir = ""
    +
    +  parse(args.toList)
    +
    +  private def parse(args: List[String]): Unit = {
    +    args match {
    +      case ("--dir" | "-d") :: value :: tail =>
    +        logDir = value
    +        parse(tail)
    +
    +      case ("--help" | "-h") :: tail =>
    +        printUsageAndExit(0)
    +
    +      case Nil =>
    +
    +      case _ =>
    +        printUsageAndExit(1)
    +    }
    +    validateLogDir()
    +  }
    +
    +  private 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))
    --- End diff --
    
    Very minor - but I'm guessing the first error condition will happen way more often than the second - and getting a message that says it's "invalid" is a lot less clear than just saying it doesn't exist. What about breaking these out and including more specific error messages in either case? Also it might be nice to print the value of `fileSystem` in the error message, so it knows what filesystem we inferred the path to be from.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39919584
  
    Also, to update others who are following this conversation. The exception reported in [SPARK-1407](https://issues.apache.org/jira/browse/SPARK-1407) turned out to be due to not properly finishing the application (i.e. through the shutdown hook triggered by System.exit without `sc.stop()`). This is not a bug inherent in the event logging interface or the HistoryServer, AFAIK.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39937975
  
     Build triggered. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39242381
  
    thanks @pwendell.   I'm going to try to look at this more detail in the next day or so.
    
    The MapReduce history server would be one thing to compare to.  It has one directory (done_intermediate) with sticky bit set where users write the history files to with the permissions specified by the user (generally restrictive).    The History Server runs as a super user and copies the history files from that done_intermediate to a done directory that is more restrictive so the world can't read/write to it.  The history server serves up the files and restricts based on acls. 
    
    The important thing is that we make it so it can be secured and document how users do that.  If its manually create some directories and set permissions I think that is fine for now.   If Spark is creating directories we need to make sure it does the right thing or has configs so that admins can have it set the permissions appropriately.
    
    Is there any infrastructure in place to manage/delete the log files?  If you are running thousands of applications a day the logs can add up pretty quickly.
    
    Can we add docs about the history server?  
    
    This is probably a separate jira, but it would be nice to clarify the documentation of config spark.eventLog.dir to indicate if it can go to hdfs or other filesystems.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39152688
  
    Build started. Build is starting -or- tests failed to complete.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11471030
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala ---
    @@ -0,0 +1,82 @@
    +/*
    + * 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.ui.{UIUtils, WebUI}
    +
    +private[spark] class IndexPage(parent: HistoryServer) {
    +
    +  def render(request: HttpServletRequest): Seq[Node] = {
    +    val appRows = parent.appIdToInfo.values.toSeq.sortBy { app => -app.lastUpdated }
    +    val appTable = UIUtils.listingTable(appHeader, appRow, appRows)
    +    val content =
    +      <div class="row-fluid">
    +        <div class="span12">
    +          <ul class="unstyled">
    +            <li><strong>Event Log Location: </strong> {parent.baseLogDir}</li>
    +          </ul>
    +          {
    +            if (parent.appIdToInfo.size > 0) {
    +              <h4>
    +                Showing {parent.appIdToInfo.size}/{parent.getNumApplications}
    +                Finished Application{if (parent.getNumApplications > 1) "s" else ""}
    --- End diff --
    
    Still says "Finished" - this is one of the more important cases because it's being shown to users and is inconsistent the other UI's that say "Completed Applications".


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40053712
  
    Jenkins, retest this please (sorry I rebooted jenkins).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40060333
  
    Merged build finished. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39454420
  
    First of all I would like to say great work, having this history server is awesome!!
    
    I'm trying it out on a non-secure yarn cluster but when I run something that accesses hdfs (SparkHdfsLR) the application ends up failing because HDFS has already been closed when it goes to logEvent.   I'll file a separate jira for that since its not directly related to this PR.  But this also causes exceptions in the history server, where once it sees something it doesn't like it throws exception and stops reading the directories  (even the good ones)
    
    14/04/03 14:01:18 ERROR history.HistoryServer: Unable to synchronize HistoryServer with files on disk:
    java.io.IOException: Filesystem closed
            at org.apache.hadoop.hdfs.DFSClient.checkOpen(DFSClient.java:398)
            at org.apache.hadoop.hdfs.DFSClient.listPaths(DFSClient.java:1284)
            at org.apache.hadoop.hdfs.DFSClient.listPaths(DFSClient.java:1269)
            at org.apache.hadoop.hdfs.DistributedFileSystem.listStatus(DistributedFileSystem.java:358)
            at org.apache.spark.deploy.history.HistoryServer.org$apache$spark$deploy$history$HistoryServer$$getModificationTime(HistoryServer.scala:171)
            at org.apache.spark.deploy.history.HistoryServer$$anonfun$1$$anonfun$apply$mcV$sp$1.apply(HistoryServer.scala:107)
            at org.apache.spark.deploy.history.HistoryServer$$anonfun$1$$anonfun$apply$mcV$sp$1.apply(HistoryServer.scala:104)
            at scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
            at scala.collection.mutable.WrappedArray.foreach(WrappedArray.scala:34)
            at org.apache.spark.deploy.history.HistoryServer$$anonfun$1.apply$mcV$sp(HistoryServer.scala:104)
            at org.apache.spark.deploy.history.HistoryServer$$anonfun$1.apply(HistoryServer.scala:99)
            at org.apache.spark.deploy.history.HistoryServer$$anonfun$1.apply(HistoryServer.scala:99)
            at scala.concurrent.impl.Future$PromiseCompletingRunnable.liftedTree1$1(Future.scala:24)
            at scala.concurrent.impl.Future$PromiseCompletingRunnable.run(Future.scala:24)
            at scala.concurrent.impl.ExecutionContextImpl$$anon$3.exec(ExecutionContextImpl.scala:107)
            at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
            at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
            at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
            at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39945706
  
    Merged build finished. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40050720
  
    
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/13988/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40011822
  
     Merged build triggered. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39940525
  
    All automated tests passed.
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/13942/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39853553
  
    For some reason I seem to have to refresh my browser twice in order to pick up new applications.  Have you seen this or is it just something funky with my browser?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11317176
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -0,0 +1,221 @@
    +/*
    + * 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 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
    +
    +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.{ApplicationEventListener, ReplayListenerBus}
    +
    +/**
    + * 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. 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)
    +  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 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
    +
    +  private val handlers = Seq[ServletContextHandler](
    +    createStaticHandler(HistoryServer.STATIC_RESOURCE_DIR, "/static"),
    +    createServletHandler("/",
    +      (request: HttpServletRequest) => indexPage.render(request), securityMgr = securityManager)
    +  )
    +
    +  val appIdToInfo = mutable.HashMap[String, ApplicationHistoryInfo]()
    +
    +  /** 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 bind HistoryServer", e)
    +        System.exit(1)
    +    }
    +    checkForLogs()
    +  }
    +
    +  /**
    +   * 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
    +   * information for this application.
    +   *
    +   * 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 {
    +        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)
    +          }
    +        }
    +      }
    +      asyncCheck.onFailure { case t =>
    +        logError("Unable to synchronize HistoryServer with files on disk: ", t)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * 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(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 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) {
    +        attachUI(ui)
    +        val appName = if (appListener.applicationStarted) appListener.appName else appId
    +        ui.setAppName("%s (finished)".format(appName))
    +        val startTime = appListener.startTime
    +        val endTime = appListener.endTime
    +        val info = ApplicationHistoryInfo(appName, startTime, endTime, lastUpdated, logPath, ui)
    --- End diff --
    
    For sure. Each SparkUI maintains many internal data structures, and it would be good to limit the number of UI's we want to keep around.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40038199
  
    All automated tests passed.
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/13974/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11434671
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -0,0 +1,322 @@
    +/*
    + * 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.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.{ApplicationEventListener, ReplayListenerBus}
    +
    +/**
    + * 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. 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)
    +  extends SparkUIContainer("History Server") with Logging {
    +
    +  import 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 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 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"),
    +    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]()
    +
    +  /**
    +   * 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. */
    +  override def bind() {
    +    try {
    +      serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, conf))
    +      logInfo("Started HistoryServer at http://%s:%d".format(publicHost, boundPort))
    +    } catch {
    +      case e: Exception =>
    +        logError("Failed to bind HistoryServer", e)
    +        System.exit(1)
    +    }
    +    checkForLogs()
    +  }
    +
    +  /**
    +   * 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
    +   * information for this application.
    +   *
    +   * If the logs for an existing finished application are no longer found, the server
    +   * removes all associated information and detaches the SparkUI.
    +   */
    +  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 mostRecentAppIds = logDirs.map { dir => getAppId(dir.getPath.toString) }
    +        appIdToInfo.foreach { case (appId, info) =>
    +          if (!mostRecentAppIds.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)
    +          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)
    +      }
    +    } else {
    +      logWarning("Attempted to check for event log updates before binding the server.")
    +    }
    +  }
    +
    +  /**
    +   * 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(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
    +    }
    +  }
    +
    +  /** Stop the server and close the file system. */
    +  override def stop() {
    +    super.stop()
    +    stopped = true
    +    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
    +
    +  /** Return when this directory was last modified. */
    +  private def getModificationTime(dir: FileStatus): Long = {
    +    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
    +    }
    +  }
    +}
    +
    +/**
    + * 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 {
    +  private val conf = new SparkConf
    +
    +  // Interval between each check for event log updates
    +  val UPDATE_INTERVAL_MS = conf.getInt("spark.history.updateInterval", 10) * 1000
    --- End diff --
    
    we need to document the new configs.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39919830
  
    Also @tgraves, I have not tried to render thousands of apps, but from what I've experienced on the SparkUI's stage page, which may display thousands of tasks, loading the HTML becomes much slower. In other words, as you suspected, the UI doesn't scale super well. I think the correct thing to do here is to add pagination (which may or may not go out with this PR).
    
    We can set the retained applications higher. I started it at 20 just because I was testing locally. The thing is each ApplicationHistoryInfo may very widely across applications, because each of them contains a SparkUI, which holds various data structures whose sizes scale with the number of tasks. For now, however, we will rotate on the granularity of applications, because this is much simpler.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40051268
  
    Merged build finished. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-38344039
  
     Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11455850
  
    --- Diff: core/src/main/scala/org/apache/spark/SparkContext.scala ---
    @@ -1115,6 +1114,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, sparkUser))
    +  }
    +
    +  /**
    +   * 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() {
    --- End diff --
    
    Doesn't this create a different race condition where there are events in the queue that get dropped because of this?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39940470
  
    Merged build started. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39449932
  
    Its not readable by other users in the intermediate directory.  Sorry I didn't explain fully.  That directory itself is 1777, sticky bit set, world writable.  Each user creates a subdirectory in there that is only readable by them and superuser group, history files are then written to that user directory with permissions so that only that user and group can read/write.
    
    The reasons for it moving it to the done directory are more for management reasons.  It orders them by date,  and then within a date makes sure one directory doesn't get to big, deletes them after a retention period (having them ordered by date makes that simple), etc.  



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11394897
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -0,0 +1,265 @@
    +/*
    + * 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.collection.mutable
    +import scala.concurrent._
    +import scala.concurrent.ExecutionContext.Implicits.global
    +
    +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.{ApplicationEventListener, ReplayListenerBus}
    +
    +/**
    + * 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. 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)
    +  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 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 val handlers = Seq[ServletContextHandler](
    +    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 {
    +      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)
    +        System.exit(1)
    +    }
    +    checkForLogs()
    +  }
    +
    +  /**
    +   * 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
    +   * information for this application.
    +   *
    +   * 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 {
    +        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) && !appIdBlacklist.contains(appId)) {
    +            maybeRenderUI(appId, path, lastUpdated)
    +          }
    +          // If the cap is reached, remove the least recently updated application
    +          if (appIdToInfo.size > RETAINED_APPLICATIONS) {
    +            removeOldestApp()
    +          }
    +        }
    +      }
    +      asyncCheck.onFailure { case t =>
    +        logError("Unable to synchronize HistoryServer with files on disk: ", t)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * 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(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
    +        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))
    +    }
    +  }
    +
    +  /** 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: 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 = {
    +    val logFiles = fileSystem.listStatus(dir.getPath)
    +    if (logFiles != null) {
    +      logFiles.map(_.getModificationTime).max
    +    } else {
    +      dir.getModificationTime
    --- End diff --
    
    need to catch exceptions here also, or perhaps just catch it higher up in checkForLogs.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39981961
  
    Other then adding a bit of documentation the changes look good to me.  I tested it out and things are working well on spark on yarn.  
    
    Note that in the future you will have to be careful about changing the format of the history files or having the readers handle changes in order keep backwards compatibility.  
    
    Also, the history server will require changes to support kerberos (reading from keytab file) so that a super user could run the history server and read from hdfs without worrying about kinit'ing every 24 hours.
    
    I'm also interested to make it work with failed/killed applications.  Note that I tried to force it to work by just touching a APPLICATION_COMPLETE file bug it fails with "Skipping incomplete application".


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39945708
  
    
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/13946/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39701638
  
    Merged build finished. All automated tests passed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40051022
  
    I'm guessing it's something random. I'll merge this as soon as it passes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11469601
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -0,0 +1,287 @@
    +/*
    + * 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.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.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.ui.JettyUtils._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * A web server that renders SparkUIs of finished applications.
    --- End diff --
    
    In Spark we tend to use the word "completed" rather than "finished" in cases like this (events like `onStageCompleted`, the other UI's which say "Completed applications". This patch uses the word "finished" in most cases. I realize this seems extremely pedantic, but having a consistent terminology would be useful here.
    
    How would you feel about replacing "finished" with "completed" throughout this patch? Are you attached to "finished"?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40039308
  
     Merged build triggered. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11469229
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -0,0 +1,287 @@
    +/*
    + * 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.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.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.ui.JettyUtils._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * 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. 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
    + */
    +class HistoryServer(
    +    val baseLogDir: String,
    +    conf: SparkConf)
    +  extends SparkUIContainer("History Server") with Logging {
    +
    +  import 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 = WEB_UI_PORT
    +  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 lastLogCheckTime = -1L
    +
    +  // Number of complete applications found in this directory
    +  private var numApplicationsTotal = 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.
    +   *
    +   * TODO: Add a mechanism to update manually.
    +   */
    +  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"),
    +    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]()
    +
    +  /**
    +   * 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. */
    +  override def bind() {
    +    try {
    +      serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, conf))
    +      logInfo("Started HistoryServer at http://%s:%d".format(publicHost, boundPort))
    +    } catch {
    +      case e: Exception =>
    +        logError("Failed to bind HistoryServer", e)
    +        System.exit(1)
    +    }
    +  }
    +
    +  /**
    +   * 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
    +   * information for this application.
    +   *
    +   * If the logs for an existing finished application are no longer found, the server
    +   * removes all associated information and detaches the SparkUI.
    +   */
    +  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]()
    +        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 }
    +
    +        // Remove any applications that should no longer be retained
    +        appIdToInfo.foreach { case (appId, info) =>
    +          if (!retainedAppIds.contains(appId)) {
    +            detachUI(info.ui)
    +            appIdToInfo.remove(appId)
    +          }
    +        }
    +
    +        // 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)) {
    +            renderSparkUI(dir, info)
    +          }
    +        }
    +
    +        // 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)
    +      }
    +    } else {
    +      logWarning("Attempted to check for event log updates before binding the server.")
    +    }
    +  }
    +
    +  /**
    +   * 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 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)
    +    }
    +  }
    +
    +  /** Stop the server and close the file system. */
    +  override def stop() {
    +    super.stop()
    +    stopped = true
    +    fileSystem.close()
    +  }
    +
    +  /** 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. */
    --- End diff --
    
    This should say completed applications, right now it's not clear whether or not this would include completed ones. Also it might be nice to say "whether or not the UI is currently rendered" instead of use the word "retained" whose meaning is less obvious.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11469982
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -0,0 +1,287 @@
    +/*
    + * 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
    --- End diff --
    
    Yeah @tdas and I are getting the same error too. Something about Intellij and Jetty 8 not being good friends.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/204


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-38507695
  
    Merged build finished.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40050917
  
     Merged build triggered. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11467806
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -0,0 +1,322 @@
    +/*
    + * 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.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.{ApplicationEventListener, ReplayListenerBus}
    +
    +/**
    + * 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. 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)
    +  extends SparkUIContainer("History Server") with Logging {
    +
    +  import 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 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 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"),
    +    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]()
    +
    +  /**
    +   * 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. */
    +  override def bind() {
    +    try {
    +      serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, conf))
    +      logInfo("Started HistoryServer at http://%s:%d".format(publicHost, boundPort))
    +    } catch {
    +      case e: Exception =>
    +        logError("Failed to bind HistoryServer", e)
    +        System.exit(1)
    +    }
    +    checkForLogs()
    +  }
    +
    +  /**
    +   * Check for any updates to event logs in the base directory. This is only effective once
    --- End diff --
    
    The newest changes involve decoupling the following two tasks:
    1) Checking whether an application is complete
    2) Replaying its event logs
    
    These were both done in ReplayListenerBus, which adds complexity when we try to do them separately in HistoryServer. The new code migrates the logic in (1) to EventLoggingListener, which is more sensible since the log formats are defined by EventLoggingListener in the first place.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11426437
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -0,0 +1,265 @@
    +/*
    + * 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.collection.mutable
    +import scala.concurrent._
    +import scala.concurrent.ExecutionContext.Implicits.global
    +
    +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.{ApplicationEventListener, ReplayListenerBus}
    +
    +/**
    + * 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. 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)
    +  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 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 val handlers = Seq[ServletContextHandler](
    +    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 {
    +      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)
    +        System.exit(1)
    +    }
    +    checkForLogs()
    +  }
    +
    +  /**
    +   * 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
    +   * information for this application.
    +   *
    +   * 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 {
    +        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) && !appIdBlacklist.contains(appId)) {
    +            maybeRenderUI(appId, path, lastUpdated)
    +          }
    +          // If the cap is reached, remove the least recently updated application
    +          if (appIdToInfo.size > RETAINED_APPLICATIONS) {
    +            removeOldestApp()
    +          }
    +        }
    +      }
    +      asyncCheck.onFailure { case t =>
    +        logError("Unable to synchronize HistoryServer with files on disk: ", t)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * 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(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
    +        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))
    +    }
    +  }
    +
    +  /** 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: 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 = {
    +    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)
    +    }
    +  }
    +
    +  /** 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
    + * 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 {
    +  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)
    +
    +  // How many applications to retain
    +  private val RETAINED_APPLICATIONS = conf.getInt("spark.deploy.retainedApplications", 20)
    --- End diff --
    
    As a rough baseline, a O(1000-task) application takes up about 5-10MB (measured using the SizeEsimator)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40060334
  
    
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/13995/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39849170
  
    I realized I'm unfortunately still hitting SPARK-1407 also.   I'll look at the issue running on secure hdfs this morning.   I haven't reviewed the changes to the Listener code in detail, perhaps Patrick or someone more familiar who did the first review could take a look also.
    
    A few suggestions and we can file these as separate jiras I think.
      - It would be nice to have a link back to the history server when you navigate down into a particular application. 
      - Adding a search box on the index table might be nice.  As the history server gets thousands of applications being able to search for something becomes useful.  For now sorting by column works. 
     -  I'm curious to see how well the html table scales.  if it takes a long time to load with lots of applications we might add some pagination/ajax type stuff.    We could also add gzip filter in to compress the data.  Have you tried rendering thousands of apps?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11262132
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -0,0 +1,221 @@
    +/*
    + * 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 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
    +
    +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.{ApplicationEventListener, ReplayListenerBus}
    +
    +/**
    + * 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. 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)
    +  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())
    --- End diff --
    
    Hey @andrewor14 - this shouldn't try to bind to SPARK_PUBLIC_DNS (good catch @tgravescs). It's sufficient to just bind to `Utils.localHostName`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40045815
  
    Merged build started. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40053228
  
    Merged build finished. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40046079
  
    This LGTM pending the small fix of changing Finished --> Completed in the UI. Is there anything else  you want to do here or is this okay to merge?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39761686
  
    This is ready for further review.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40042754
  
    Merged build finished. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40062920
  
    All automated tests passed.
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/13999/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11459225
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -0,0 +1,322 @@
    +/*
    + * 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.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.{ApplicationEventListener, ReplayListenerBus}
    +
    +/**
    + * 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. 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)
    +  extends SparkUIContainer("History Server") with Logging {
    +
    +  import 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 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 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
    --- End diff --
    
    oops - sorry I see the TODO


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11469925
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -90,9 +97,115 @@ 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)
    +
    +  /**
    +   * Stop logging events.
    +   * In addition, create an empty special file to indicate application completion.
    +   */
    +  def stop() = {
    +    logger.newFile(APPLICATION_COMPLETE)
    +    logger.stop()
    +  }
    +}
    +
    +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 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)
    +  }
     
    -  def stop() = logger.stop()
    +  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, "")
    +    } 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("<Unknown>"),
    +        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)
    --- End diff --
    
    Good point


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11393951
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -0,0 +1,265 @@
    +/*
    + * 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.collection.mutable
    +import scala.concurrent._
    +import scala.concurrent.ExecutionContext.Implicits.global
    +
    +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.{ApplicationEventListener, ReplayListenerBus}
    +
    +/**
    + * 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. 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)
    +  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 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 val handlers = Seq[ServletContextHandler](
    +    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 {
    +      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)
    +        System.exit(1)
    +    }
    +    checkForLogs()
    +  }
    +
    +  /**
    +   * 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
    +   * information for this application.
    +   *
    +   * 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 {
    +        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) && !appIdBlacklist.contains(appId)) {
    +            maybeRenderUI(appId, path, lastUpdated)
    +          }
    +          // If the cap is reached, remove the least recently updated application
    +          if (appIdToInfo.size > RETAINED_APPLICATIONS) {
    +            removeOldestApp()
    +          }
    +        }
    +      }
    +      asyncCheck.onFailure { case t =>
    +        logError("Unable to synchronize HistoryServer with files on disk: ", t)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * 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(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
    +        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))
    +    }
    +  }
    +
    +  /** 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: 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 = {
    +    val logFiles = fileSystem.listStatus(dir.getPath)
    --- End diff --
    
    We should catch exceptions here.  For instances if for some reason there is a directory the history server user doesn't have access to it will throw a AccessControlException. This then prevents any applications after it in the directory from being displayed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40053913
  
    Merged build started. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40044096
  
     Merged build triggered. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40042755
  
    
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/13976/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-38345705
  
    Merged build finished.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11253731
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -0,0 +1,221 @@
    +/*
    + * 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 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
    +
    +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.{ApplicationEventListener, ReplayListenerBus}
    +
    +/**
    + * 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. 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)
    +  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 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
    +
    +  private val handlers = Seq[ServletContextHandler](
    +    createStaticHandler(HistoryServer.STATIC_RESOURCE_DIR, "/static"),
    +    createServletHandler("/",
    +      (request: HttpServletRequest) => indexPage.render(request), securityMgr = securityManager)
    +  )
    +
    +  val appIdToInfo = mutable.HashMap[String, ApplicationHistoryInfo]()
    +
    +  /** Bind to the HTTP server behind this web interface */
    +  override def bind() {
    +    try {
    +      serverInfo = Some(startJettyServer(host, port, handlers, conf))
    --- End diff --
    
    does this have the same problem Patrick just fixed in SPARK-1324 with binding to public dns


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11389436
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala ---
    @@ -0,0 +1,76 @@
    +/*
    + * 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.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 =
    +      <div class="row-fluid">
    +        <div class="span12">
    +          <ul class="unstyled">
    +            <li><strong>Event Log Location: </strong> {parent.baseLogDir}</li>
    +            <h4>
    +              Showing {parent.appIdToInfo.size}/{parent.getTotalApplications} Finished Applications
    +            </h4>
    +            {appTable}
    +          </ul>
    +        </div>
    +      </div>
    +
    +    UIUtils.basicSparkPage(content, "History Server")
    +  }
    +
    +  private val appHeader = Seq(
    +    "App Name",
    +    "Started",
    +    "Finished",
    +    "Duration",
    +    "Log Directory",
    +    "Last Updated")
    +
    +  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) 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)
    --- End diff --
    
    It would be nice to display a column with the user name also.  I expect the history server to be shared on our clusters so its hard filter out just my applications. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11389264
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -0,0 +1,265 @@
    +/*
    + * 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.collection.mutable
    +import scala.concurrent._
    +import scala.concurrent.ExecutionContext.Implicits.global
    +
    +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.{ApplicationEventListener, ReplayListenerBus}
    +
    +/**
    + * 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. 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)
    +  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 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 val handlers = Seq[ServletContextHandler](
    +    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]()
    --- End diff --
    
    This isn't ideal for scaling.  At some point this is going to get huge unless the application history logs are cleaned up.  Perhaps we leave that for a follow on improvement though since I think you are limited by the way its storing them now.  The first thing that came to mind was just to list by date and choose the latest x but hadoop filesystem doesn't give you that filter without doing stat on each one afterwards. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39156082
  
    Merged build finished. Build is starting -or- tests failed to complete.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11332376
  
    --- Diff: 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 <base-log-dir> [<web-ui-port>]
    +#   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 <base-log-dir> [<web-ui-port>]"
    +  echo "Example: ./start-history-server.sh /tmp/spark-events 18080"
    --- End diff --
    
    In the latest commit, the history server reads from SPARK_DAEMON_MEMORY the same way Masters and Workers do. This may or may not be subject to change with #299.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39790271
  
    Thanks @tgravescs. What is the exception?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11456474
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -0,0 +1,322 @@
    +/*
    + * 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.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.{ApplicationEventListener, ReplayListenerBus}
    +
    +/**
    + * 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. 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)
    +  extends SparkUIContainer("History Server") with Logging {
    +
    +  import 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 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 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"),
    +    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]()
    +
    +  /**
    +   * 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. */
    +  override def bind() {
    +    try {
    +      serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, conf))
    +      logInfo("Started HistoryServer at http://%s:%d".format(publicHost, boundPort))
    +    } catch {
    +      case e: Exception =>
    +        logError("Failed to bind HistoryServer", e)
    +        System.exit(1)
    +    }
    +    checkForLogs()
    +  }
    +
    +  /**
    +   * 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
    +   * information for this application.
    +   *
    +   * If the logs for an existing finished application are no longer found, the server
    +   * removes all associated information and detaches the SparkUI.
    +   */
    +  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 mostRecentAppIds = logDirs.map { dir => getAppId(dir.getPath.toString) }
    --- End diff --
    
    I found the name `mostRecentAppIds` a bit confusing... I would expect it to show me e.g. the N most recent applications sorted by time. But really it shows all of the applications which are currently present.
    
    What about `presentAppIds` or `foundAppIds`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39420831
  
    Hey @tgravescs, just so I understand, what purpose does the intermediate directory serve in this case? It seems like if the file is visible to other users in the intermediate directory, then the security is already broken. If the file is not visible in the intermediate directory, then copying it wont' change the visibility for other users... 
    
    I think if we just add a config to set the permissions bits on the directory Spark creates for each application, then we are in pretty good shape (provided document this as well).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39853955
  
    Do you have access to a secure hdfs cluster?  I added the stack trace about that to jira: https://issues.apache.org/jira/browse/SPARK-1407.   If you don't have access perhaps it makes more sense to fix it under that jira. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39937983
  
    Build started. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11458932
  
    --- Diff: core/src/main/scala/org/apache/spark/ui/WebUI.scala ---
    @@ -20,6 +20,25 @@ package org.apache.spark.ui
     import java.text.SimpleDateFormat
     import java.util.Date
     
    +private[spark] abstract class WebUI(name: String) {
    --- End diff --
    
    So then - what is the benefit of having it? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39155211
  
    Test failure seems unrelated: `Could not find Apache license headers in the following files:
     !????? /root/workspace/SparkPullRequestBuilder/.mima-excludes`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11471042
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala ---
    @@ -0,0 +1,82 @@
    +/*
    + * 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.ui.{UIUtils, WebUI}
    +
    +private[spark] class IndexPage(parent: HistoryServer) {
    +
    +  def render(request: HttpServletRequest): Seq[Node] = {
    +    val appRows = parent.appIdToInfo.values.toSeq.sortBy { app => -app.lastUpdated }
    +    val appTable = UIUtils.listingTable(appHeader, appRow, appRows)
    +    val content =
    +      <div class="row-fluid">
    +        <div class="span12">
    +          <ul class="unstyled">
    +            <li><strong>Event Log Location: </strong> {parent.baseLogDir}</li>
    +          </ul>
    +          {
    +            if (parent.appIdToInfo.size > 0) {
    +              <h4>
    +                Showing {parent.appIdToInfo.size}/{parent.getNumApplications}
    +                Finished Application{if (parent.getNumApplications > 1) "s" else ""}
    --- End diff --
    
    Note that this is here and also below in the UI


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40114842
  
    Thanks, merged into master and 1.0.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39701639
  
    All automated tests passed.
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/13837/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40015887
  
    All automated tests passed.
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/13957/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/204#discussion_r11470537
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -90,9 +97,115 @@ 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)
    +
    +  /**
    +   * Stop logging events.
    +   * In addition, create an empty special file to indicate application completion.
    +   */
    +  def stop() = {
    +    logger.newFile(APPLICATION_COMPLETE)
    +    logger.stop()
    +  }
    +}
    +
    +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"
    --- End diff --
    
    I meant it might be nice if we had a different file name that users could set if they manually create the file. I guess we can always infer failure by just lacking an end event, so maybe that's not necessary.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40060096
  
    Merged build started. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-40053909
  
     Merged build triggered. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-1276] Add a HistoryServer to render per...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/204#issuecomment-39155070
  
    Build started. Build is starting -or- tests failed to complete.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---