You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by vanzin <gi...@git.apache.org> on 2014/05/09 23:42:17 UTC

[GitHub] spark pull request: [SPARK-1768] History server enhancements.

GitHub user vanzin opened a pull request:

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

    [SPARK-1768] History server enhancements.

    Two improvements to the history server:
    
    - Separate the HTTP handling from history fetching, so that it's easy to add
      new backends later (thinking about SPARK-1537 in the long run)
    
    - Avoid loading all UIs in memory. Do lazy loading instead, keeping a few in
      memory for faster access. This allows the app limit to go away, since holding
      just the listing in memory shouldn't be too expensive unless the user has millions
      of completed apps in the history (at which point I'd expect other issues to arise
      aside from history server memory usage, such as FileSystem.listStatus()
      starting to become ridiculously expensive).
    
    I also fixed a few minor things along the way which aren't really worth mentioning.
    I also removed the app's log path from the UI since that information may not even
    exist depending on which backend is used (even though there is only one now).

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

    $ git pull https://github.com/vanzin/spark hist-server

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

    https://github.com/apache/spark/pull/718.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 #718
    
----
commit 60e07aed62d9c6632775ad600a8c80fc37844201
Author: Marcelo Vanzin <va...@cloudera.com>
Date:   2014-05-07T23:30:02Z

    Separate history server from history backend.
    
    This change does two things, mainly:
    
    - Separate the logic of serving application history from fetching
      application history from the underlying storage. Not only this
      cleans up the code a little bit, but it also serves as initial
      work for SPARK-1537, where we may want to fetch application data
      from Yarn instead of HDFS.
    
      I've kept the current command line options working, but I changed
      the way configuration is set to be mostly based on SparkConf,
      so that it's easy to support new providers later.
    
    - Make it so the UI for each application is loaded lazily. The
      UIs are cached in memory (cache size configurable) for faster
      subsequent access. This means that we don't need a limit for
      the number of applications listed; the list should fit
      comfortably in memory (since it holds much less data).
    
      Because of this I lowered the number of applications kept in
      memory to 50 (since that setting doesn't influence the number
      of apps listed anymore).
    
    Later, we may want to provide paging in the listing UI, and also
    spilling the listing to disk and loading it on demand to avoid
    large memory usage / slow startup.

commit 0dff2e631f367435bbb6cacd8dab79e14d657e19
Author: Marcelo Vanzin <va...@cloudera.com>
Date:   2014-05-08T21:48:04Z

    Rudimentary paging support for the history UI.
    
    The provider's list api was tweaked a little bit so that the caller
    can get an atomic view of the data currently held in the provider.

commit 286d9eb96e4fcff97ca109e46e5c72a16be7d19d
Author: Marcelo Vanzin <va...@cloudera.com>
Date:   2014-05-09T20:39:26Z

    Ensure server.stop() is called when shutting down.
    
    Also remove the cleanup code from the fs provider. It would be
    better to clean up, but there's a race between that code's cleanup
    and Hadoop's shutdown hook, which closes all file systems kept in
    the cache. So if you try to clean up the fs provider in a shut
    down hook, you may end up with ugly exceptions in the output.
    
    But leave the stop() functionality around in case it's useful for
    future provider implementations.

----


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-46267852
  
     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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13621315
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala ---
    @@ -67,10 +69,29 @@ private[spark] class HistoryServerArguments(args: Array[String]) {
     
       private def printUsageAndExit(exitCode: Int) {
         System.err.println(
    -      "Usage: HistoryServer [options]\n" +
    -      "\n" +
    -      "Options:\n" +
    -      "  -d DIR,  --dir DIR     Location of event log files")
    +      """
    +      |Usage: HistoryServer [-d logDir]
    +      |
    +      |The preferred way to pass options is to set the configuration below using
    +      |SPARK_HISTORY_OPTS. The "-d" command line argument is avalable for backwards
    --- End diff --
    
    Right now there are two ways of setting the logging directory, one through the command line and the other through `spark.history.fs.logDirectory`. In the long run, we should favor the latter, as your comments seem to suggest. Given that, I think we should just remove all documentation of the command line way, but for backward compatibility keep the code that supports it. Otherwise, it's a little confusing for the user if we document both ways. (Note that there are multiple files that document this: HistoryServer.scala, HistoryServerArguments.scala, and monitoring.md)
    
    Then,  in `start-history-server.sh`, if the user passes in a command line argument, we should print a deprecated warning, and convert that to `spark.history.fs.logDirectory` as a JVM `-D` option.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-46269737
  
    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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13630803
  
    --- Diff: docs/monitoring.md ---
    @@ -69,7 +71,13 @@ represents an application's event logs. This creates a web interface at
     <table class="table">
       <tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
       <tr>
    -    <td>spark.history.updateInterval</td>
    +    <td>spark.history.provider</td>
    +    <td>org.apache.spark.deploy.history.FsHistoryProvider</td>
    +    <td>Name of the class implementing the application history backend. Currently there is only
    +    one implementation provided by Spark, which matches the default value.</td>
    --- End diff --
    
    Maybe add a line explaining what this implementation is? Perhaps something along the lines of "... provided by Spark, which renders application UI from logs stored in the file system"


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13570852
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala ---
    @@ -25,20 +25,32 @@ import org.apache.spark.ui.{WebUIPage, UIUtils}
     
     private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") {
     
    +  val pageSize = 20
    +
       def render(request: HttpServletRequest): Seq[Node] = {
    -    val appRows = parent.appIdToInfo.values.toSeq.sortBy { app => -app.lastUpdated }
    -    val appTable = UIUtils.listingTable(appHeader, appRow, appRows)
    +    val requestedPage = Option(request.getParameter("page")).getOrElse("1").toInt
    +    val requestedFirst = (requestedPage - 1) * pageSize
    +
    +    val allApps = parent.getApplicationList()
    +    val actualFirst = if (requestedFirst < allApps.size) requestedFirst else 0
    +    val apps = allApps.slice(actualFirst, Math.min(pageSize, allApps.size))
    +
    +    val actualPage = (actualFirst / pageSize) + 1
    +    val last = Math.min(actualFirst + pageSize, allApps.size) - 1
    +    val pageCount = allApps.size / pageSize + (if (allApps.size % pageSize > 0) 1 else 0)
    +
    +    val appTable = UIUtils.listingTable(appHeader, appRow, apps)
         val content =
           <div class="row-fluid">
             <div class="span12">
    -          <ul class="unstyled">
    -            <li><strong>Event Log Location: </strong> {parent.baseLogDir}</li>
    --- End diff --
    
    I can put it back. I was thinking forward, when the backend storage might not be the file system at all. But we can remove it then.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13618784
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,210 @@
    +/*
    + * 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.io.FileNotFoundException
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval",
    +    conf.getInt("spark.history.updateInterval", 10)) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  @volatile private var appList: Seq[ApplicationHistoryInfo] = Nil
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTimeMs()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing() = appList
    +
    +  override def getAppUI(appId: String): SparkUI = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)._2
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  override def getConfig(): Map[String, String] =
    +    Map(("Event Log Location" -> logDir))
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, by not reading
    +   * applications that haven't been updated since last time the logs were checked.
    +   */
    +  private def checkForLogs() = {
    +    lastLogCheckTimeMs = getMonotonicTimeMs()
    +    logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs))
    +    try {
    +      val logStatus = fs.listStatus(new Path(logDir))
    +      val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
    +      val logInfos = logDirs
    +        .filter {
    +            dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))
    +          }
    +
    +      val currentApps = Map[String, ApplicationHistoryInfo](
    +        appList.map(app => (app.id -> app)):_*)
    +
    +      // For any application that either (i) is not listed or (ii) has changed since the last time
    +      // the listing was created (defined by the log dir's modification time), load the app's info.
    +      // Otherwise just reuse what's already in memory.
    +      val newApps = new mutable.ArrayBuffer[ApplicationHistoryInfo](logInfos.size)
    +      for (dir <- logInfos) {
    +        val curr = currentApps.getOrElse(dir.getPath().getName(), null)
    +        if (curr == null || curr.lastUpdated < getModificationTime(dir)) {
    +          try {
    +            newApps += loadAppInfo(dir, false)._1
    +          } catch {
    +            case e: Exception => logError(s"Failed to load app info from directory $dir.")
    +          }
    +        } else {
    +          newApps += curr
    +        }
    +      }
    +
    +      appList = newApps.sortBy { info => -info.endTime }
    +    } catch {
    +      case t: Throwable => logError("Exception in checking for event log updates", t)
    +    }
    +  }
    +
    +  /**
    +   * Parse the application's logs to find out the information we need to build the
    +   * listing page.
    +   *
    +   * When creating the listing of available apps, there is no need to load the whole UI for the
    +   * application. The UI is requested by the HistoryServer (by calling getAppInfo()) when the user
    +   * clicks on a specific application.
    +   *
    +   * @param logDir Directory with application's log files.
    +   * @param renderUI Whether to create the SparkUI for the application.
    +   * @return A 2-tuple `(app info, ui)`. `ui` will be null if `renderUI` is false.
    --- End diff --
    
    Looks like we're only ever interested in one of these at a time. If `renderUI` is true, we're only interested in the UI. Otherwise, we're only interested in the app info. We should abstract this so we don't have to return a tuple.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13565675
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -38,56 +39,75 @@ import org.apache.spark.util.Utils
      * 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,
    +    provider: ApplicationHistoryProvider,
         securityManager: SecurityManager,
    -    conf: SparkConf)
    -  extends WebUI(securityManager, HistoryServer.WEB_UI_PORT, conf) with Logging {
    +    port: Int)
    +  extends WebUI(securityManager, port, conf) with Logging {
    +
    +  // How many applications to retain
    +  private val retainedApplications = conf.getInt("spark.history.retainedApplications", 50)
     
    -  import HistoryServer._
    +  // set whether to enable or disable view acls for all applications
    +  private val uiAclsEnabled = conf.getBoolean("spark.history.ui.acls.enable", false)
     
    -  private val fileSystem = Utils.getHadoopFileSystem(baseLogDir)
       private val localHost = Utils.localHostName()
       private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHost)
    --- End diff --
    
    No longer 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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-45123904
  
    Hi @pwendell,
    
    First, thanks for taking the time to look into this.
    
    While it's true that this change mixes a few arguably unrelated things, I'm a little worried about breaking it up further. Mainly because github's pull request feature doesn't seem to handle incremental changes well (see my previous comment, https://github.com/apache/spark/pull/718#issuecomment-45023292). If there's a way I'm not aware of that allows me to send multiple, related PRs through github, I'll gladly make the changes. Otherwise, I'd rather keep this PR as is since I already have more work that depends on it, and breaking it down would just slow down the whole process even more.
    
    About (3), note that one of the goals of the enhancements, as I mentioned in the Jira issue, is to prepare the field for SPARK-1537. Also, the refactoring allows for easier unit testing (which I added as part of my related change mentioned in my above comment). I'm not worried about having to change the interface later - these are internal APIs, after all. But I really would like to have the UI handling separate from the history backend.
    
    About (4), the main thing is to allow arbitrary options to be set more easily. I'm not wedded to the idea of a new command line argument (see other comments), but I think we should at least add support for reading config files (not just in the History Server, but also other daemons). I think that more closely matches how people generally deploy daemons.
    
    On a side note, just in case github doesn't support the "related PRs" thing I tried to explain, I'd suggest you guys take a look at something like ReviewBoard. RB makes that pretty painless, since it's not necessarily tied to github branches. It also handles things like renaming files more cleanly (when tied to a git repo). It would probably make merging changes a little more difficult, but I'm sure that could be made better with some scripts. In any case, I'm really looking for suggestions about how to handle these related PRs - sending them one at a time and having to wait until they are committed before sending the next one really slows things considerably, and prevents people from seeing the "big picture" when work spans several changes.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-46470850
  
    Same error as before - the test passes for me locally, on top of  current master.


---
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-1768] History server enhancements.

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

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


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-46225589
  
    No changes, just rebased on top of current master.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13571301
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,193 @@
    +/*
    + * 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.io.FileNotFoundException
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval", 10) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  @volatile private var appList: Seq[ApplicationHistoryInfo] = Nil
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTime()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing() = appList
    +
    +  override def getAppInfo(appId: String): ApplicationHistoryInfo = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, by not reading
    +   * applications that haven't been updated since last time the logs were checked.
    +   */
    +  def checkForLogs() = {
    +    lastLogCheckTimeMs = getMonotonicTime()
    +    logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs))
    +    try {
    +      val logStatus = fs.listStatus(new Path(logDir))
    +      val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
    +      val logInfos = logDirs
    +        .filter {
    +            dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))
    +          }
    +
    +      val currentApps = Map[String, ApplicationHistoryInfo](
    +        appList.map(app => (app.id -> app)):_*)
    +
    +      // For any application that either (i) is not listed or (ii) has changed since the last time
    +      // the listing was created (defined by the log dir's modification time), load the app's info.
    +      // Otherwise just reuse what's already in memory.
    +      val newApps = new mutable.ArrayBuffer[ApplicationHistoryInfo](logInfos.size)
    +      for (dir <- logInfos) {
    +        val curr = currentApps.getOrElse(dir.getPath().getName(), null)
    +        if (curr == null || curr.lastUpdated < getModificationTime(dir)) {
    +          try {
    +            newApps += loadAppInfo(dir, false)
    +          } catch {
    +            case e: Exception => logError(s"Failed to load app info from directory $dir.")
    +          }
    +        } else {
    +          newApps += curr
    +        }
    +      }
    +
    +      appList = newApps.sortBy { info => -info.endTime }
    +    } catch {
    +      case t: Throwable => logError("Exception in checking for event log updates", t)
    +    }
    +  }
    +
    +  /**
    +   * Parse the application's logs to find out the information we need to build the
    +   * listing page.
    +   */
    +  private def loadAppInfo(logDir: FileStatus, renderUI: Boolean): ApplicationHistoryInfo = {
    +    val elogInfo = EventLoggingListener.parseLoggingInfo(logDir.getPath(), fs)
    +    val path = logDir.getPath
    +    val appId = path.getName
    +    val replayBus = new ReplayListenerBus(elogInfo.logPaths, fs, elogInfo.compressionCodec)
    +    val appListener = new ApplicationEventListener
    +    replayBus.addListener(appListener)
    +
    +    val ui: SparkUI = if (renderUI) {
    +        val conf = this.conf.clone()
    +        val appSecManager = new SecurityManager(conf)
    +        new SparkUI(conf, appSecManager, replayBus, appId, "/history/" + appId)
    +        // Do not call ui.bind() to avoid creating a new server for each application
    +      } else {
    +        null
    +      }
    +
    +    replayBus.replay()
    +    ApplicationHistoryInfo(appId,
    +      appListener.appName,
    +      appListener.startTime,
    +      appListener.endTime,
    +      getModificationTime(logDir),
    +      appListener.sparkUser,
    +      if (renderUI) appListener.viewAcls else null,
    +      ui)
    +  }
    +
    +  /** Return when this directory was last modified. */
    +  private def getModificationTime(dir: FileStatus): Long = {
    +    try {
    +      val logFiles = fs.listStatus(dir.getPath)
    +      if (logFiles != null && !logFiles.isEmpty) {
    +        logFiles.map(_.getModificationTime).max
    +      } else {
    +        dir.getModificationTime
    +      }
    +    } catch {
    +      case t: Throwable =>
    +        logError("Exception in accessing modification time of %s".format(dir.getPath), t)
    +        -1L
    +    }
    +  }
    +
    +  /** Returns the system's mononotically increasing time. */
    +  private def getMonotonicTime() = System.nanoTime() / (1000 * 1000)
    +
    --- End diff --
    
    The background thread is a daemon thread, so it will die with the process. There's no need to wait for it to finish what it's doing, since any data it's reading will just be discarded (and it might just hit lots of exceptions because of the shutdown hook ordering).
    
    Unless you'd prefer to register a shutdown hook using Hadoop's ShutdownHookManager to run before HDFS's, but that feels hacky for no real tangible gain (in this case).


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-46741640
  
    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-1768] History server enhancements.

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

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


---
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-1768] History server enhancements.

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

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


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-46382432
  
    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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13618643
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -38,56 +39,70 @@ import org.apache.spark.util.Utils
      * 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,
    +    provider: ApplicationHistoryProvider,
         securityManager: SecurityManager,
    -    conf: SparkConf)
    -  extends WebUI(securityManager, HistoryServer.WEB_UI_PORT, conf) with Logging {
    +    port: Int)
    +  extends WebUI(securityManager, port, conf) with Logging {
     
    -  import HistoryServer._
    +  // How many applications to retain
    +  private val retainedApplications = conf.getInt("spark.history.retainedApplications", 50)
     
    -  private val fileSystem = Utils.getHadoopFileSystem(baseLogDir)
       private val localHost = Utils.localHostName()
    -  private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHost)
     
    -  // A timestamp of when the disk was last accessed to check for log updates
    -  private var lastLogCheckTime = -1L
    +  private val appLoader = new CacheLoader[String, SparkUI] {
    +    override def load(key: String): SparkUI = {
    +      val ui = provider.getAppUI(key)
    +      if (ui == null) {
    +        throw new NoSuchElementException()
    +      }
    +      attachSparkUI(ui)
    +      ui
    +    }
    +  }
     
    -  // Number of completed applications found in this directory
    -  private var numCompletedApplications = 0
    +  private val appCache = CacheBuilder.newBuilder()
    +    .maximumSize(retainedApplications)
    +    .removalListener(new RemovalListener[String, SparkUI] {
    +      override def onRemoval(rm: RemovalNotification[String, SparkUI]) = {
    +        detachSparkUI(rm.getValue())
    +      }
    +    })
    +    .build(appLoader)
    +
    +  private val loaderServlet = new HttpServlet {
    +    protected override def doGet(req: HttpServletRequest, res: HttpServletResponse): Unit = {
    +      val parts = Option(req.getPathInfo()).getOrElse("").split("/")
    +      if (parts.length < 2) {
    +        res.sendError(HttpServletResponse.SC_BAD_REQUEST,
    +          s"Unexpected path info in request (URI = ${req.getRequestURI()}")
    --- End diff --
    
    Could you also `logWarning` this? I don't think this currently shows up in the 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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13398280
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,203 @@
    +/*
    + * 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.io.FileNotFoundException
    +import java.util.concurrent.atomic.AtomicReference
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval", 10) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  private val appList = new AtomicReference[Seq[ApplicationHistoryInfo]](Nil)
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTime()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing(offset: Int, count: Int) = {
    +    val list = appList.get()
    +    val theOffset = if (offset < list.size) offset else 0
    +    (list.slice(theOffset, Math.min(theOffset + count, list.size)), theOffset, list.size)
    +  }
    +
    +  override def getAppInfo(appId: String): ApplicationHistoryInfo = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, but not reading
    +   * applications that hasn't been updated since last time the logs were checked.
    +   */
    +  def checkForLogs() = synchronized {
    --- End diff --
    
    No need for synchronization, since there's only one update thread.


---
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-1768] History server enhancements.

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

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


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13873985
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala ---
    @@ -0,0 +1,60 @@
    +/*
    + * 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 org.apache.spark.ui.SparkUI
    +
    +private[spark] case class ApplicationHistoryInfo(
    +    id: String,
    +    name: String,
    +    startTime: Long,
    +    endTime: Long,
    +    lastUpdated: Long,
    +    sparkUser: String)
    +
    +private[spark] abstract class ApplicationHistoryProvider {
    +
    +  /**
    +   * This method should return a list of applications available for the history server to
    --- End diff --
    
    Minor (I can fix this on merge). But these should not start with "This method should...". That's implied by the fact that this is an abstract class.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-46327284
  
    Guess I need to figure out how to exclude HistoryServer from compatibility checks.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13667253
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,210 @@
    +/*
    + * 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.io.FileNotFoundException
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval",
    +    conf.getInt("spark.history.updateInterval", 10)) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  @volatile private var appList: Seq[ApplicationHistoryInfo] = Nil
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTimeMs()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing() = appList
    +
    +  override def getAppUI(appId: String): SparkUI = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)._2
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  override def getConfig(): Map[String, String] =
    +    Map(("Event Log Location" -> logDir))
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, by not reading
    +   * applications that haven't been updated since last time the logs were checked.
    +   */
    +  private def checkForLogs() = {
    +    lastLogCheckTimeMs = getMonotonicTimeMs()
    +    logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs))
    +    try {
    +      val logStatus = fs.listStatus(new Path(logDir))
    +      val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
    +      val logInfos = logDirs
    +        .filter {
    +            dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))
    +          }
    +
    +      val currentApps = Map[String, ApplicationHistoryInfo](
    +        appList.map(app => (app.id -> app)):_*)
    +
    +      // For any application that either (i) is not listed or (ii) has changed since the last time
    +      // the listing was created (defined by the log dir's modification time), load the app's info.
    +      // Otherwise just reuse what's already in memory.
    +      val newApps = new mutable.ArrayBuffer[ApplicationHistoryInfo](logInfos.size)
    +      for (dir <- logInfos) {
    +        val curr = currentApps.getOrElse(dir.getPath().getName(), null)
    +        if (curr == null || curr.lastUpdated < getModificationTime(dir)) {
    +          try {
    +            newApps += loadAppInfo(dir, false)._1
    +          } catch {
    +            case e: Exception => logError(s"Failed to load app info from directory $dir.")
    +          }
    +        } else {
    +          newApps += curr
    +        }
    +      }
    +
    +      appList = newApps.sortBy { info => -info.endTime }
    +    } catch {
    +      case t: Throwable => logError("Exception in checking for event log updates", t)
    +    }
    +  }
    +
    +  /**
    +   * Parse the application's logs to find out the information we need to build the
    +   * listing page.
    +   *
    +   * When creating the listing of available apps, there is no need to load the whole UI for the
    +   * application. The UI is requested by the HistoryServer (by calling getAppInfo()) when the user
    +   * clicks on a specific application.
    +   *
    +   * @param logDir Directory with application's log files.
    +   * @param renderUI Whether to create the SparkUI for the application.
    +   * @return A 2-tuple `(app info, ui)`. `ui` will be null if `renderUI` is false.
    --- End diff --
    
    At the least, we should move the UI view acls code up, so we don't have to do an extra null check.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13630475
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala ---
    @@ -0,0 +1,61 @@
    +/*
    + * 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 org.apache.spark.ui.SparkUI
    +
    +private[spark] case class ApplicationHistoryInfo(
    +    id: String,
    +    name: String,
    +    startTime: Long,
    +    endTime: Long,
    +    lastUpdated: Long,
    +    sparkUser: String) {
    +}
    --- End diff --
    
    nit: no need to add `{ }` for case class


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13404251
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -290,8 +217,88 @@ object HistoryServer {
         }
       }
     
    +  private def parse(args: List[String]): Unit = {
    +    args match {
    +      case ("--dir" | "-d") :: value :: tail =>
    +        set("fs.logDirectory",  value)
    +        parse(tail)
    +
    +      case ("--port" | "-p") :: value :: tail =>
    +        set("ui.port", value)
    +        parse(tail)
    +
    +      case ("-D") :: opt :: value :: tail =>
    +        set(opt, value)
    +        parse(tail)
    +
    +      case ("--help" | "-h") :: tail =>
    +        printUsageAndExit(0)
    +
    +      case Nil =>
    +
    +      case _ =>
    +        printUsageAndExit(1)
    +    }
    +  }
    +
    +  private def set(name: String, value: String) = {
    +    conf.set("spark.history." + name, value)
    +  }
    +
    +  private def printUsageAndExit(exitCode: Int) {
    +    System.err.println(
    +      """
    +      |Usage: HistoryServer [options]
    +      |
    +      |Options are set by passing "-D option value" command line arguments to the class.
    +      |Command line options will override the Spark configuration file and system properties.
    +      |History Server options are always available; additional options depend on the provider.
    +      |
    +      |History Server options:
    +      |
    +      |  ui.port           Port where server will listen for connections (default 18080)
    +      |  ui.acls.enable    Whether to enable view acls for all applications (default false)
    +      |  provider          Name of history provider class (defaults to file system-based provider)
    +      |
    +      |FsHistoryProvider options:
    +      |
    +      |  fs.logDirectory   Directory where app logs are stored (required)
    +      |  fs.updateInterval How often to reload log data from storage (seconds, default 10)
    +      |""".stripMargin)
    +    System.exit(exitCode)
    +  }
    +
     }
     
    +private[spark] abstract class ApplicationHistoryProvider {
    +
    +  /**
    +   * This method should return a list of applications available for the history server to
    +   * show. The listing is assumed to be in descending time order.
    +   *
    +   * An adjusted offset should be returned if the app list has changed and the request
    +   * references an invalid start offset. Otherwise, the provided offset should be returned.
    +   *
    +   * @param offset Starting offset for returned objects.
    +   * @param count Max number of objects to return.
    +   * @return 3-tuple (requested app list, adjusted offset, count of all available apps)
    +   */
    +  def getListing(offset: Int, count: Int): (Seq[ApplicationHistoryInfo], Int, Int)
    --- End diff --
    
    I'm not sure I totally follow. If you mean both methods return the same type (ApplicationHistoryInfo) I agree that's a little bit confusing. It could be easily changed though. I can clarify what the ordering means, pending the rest of the discussion.
    
    Regarding predicates, I was deliberately avoiding going down that path. I don't see a good way to have both the FS-based backend and a future Yarn backend cleanly sharing the same predicate language. Doing so would mean creating some Spark-specific language for that, and parsing / translating that to what yarn understands. Or supporting a subset of Yarn's parameters in the non-Yarn backend. 
    
    I'm not sure that's the best path forward (although I understand not everybody uses Yarn, let alone the latest and greatest version). In my view, when using Yarn as the backend, the user would navigate the listing using Yarn's UI, and that would link to the Spark history server for rendering individual applications. The SHS listing page would be just a simple fallback if someone ends up going there, but wouldn't provide many features, and having offset / limit parameters would be the bare minimum to allow the server to scale and at the same time not flood the client with a huge HTML page.
    
    But if you think that the SHS should be enhanced in the future to support these kinds of predicates, offset / limits can be built into that language.
    
    BTW, this is what the Yarn API exposes as far as predicates:
    
          @PathParam("entityType") String entityType,
          @QueryParam("primaryFilter") String primaryFilter,
          @QueryParam("secondaryFilter") String secondaryFilter,
          @QueryParam("windowStart") String windowStart,
          @QueryParam("windowEnd") String windowEnd,
          @QueryParam("fromId") String fromId,
          @QueryParam("fromTs") String fromTs,
          @QueryParam("limit") String limit,
          @QueryParam("fields") String fields
    
    I think you could map "offset" to the windowing parameters, although documentation there is still kinda lacking. Anyway, to avoid going down a rat hole here, I'll just pull the offset / limit parameters into the UI layer for 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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13365862
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,203 @@
    +/*
    + * 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.io.FileNotFoundException
    +import java.util.concurrent.atomic.AtomicReference
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval", 10) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  private val appList = new AtomicReference[Seq[ApplicationHistoryInfo]](Nil)
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTime()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing(offset: Int, count: Int) = {
    +    val list = appList.get()
    +    val theOffset = if (offset < list.size) offset else 0
    +    (list.slice(theOffset, Math.min(theOffset + count, list.size)), theOffset, list.size)
    +  }
    +
    +  override def getAppInfo(appId: String): ApplicationHistoryInfo = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, but not reading
    +   * applications that hasn't been updated since last time the logs were checked.
    --- End diff --
    
    should be "not reading applications that haven't been updated"


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13665410
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,210 @@
    +/*
    + * 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.io.FileNotFoundException
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval",
    +    conf.getInt("spark.history.updateInterval", 10)) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  @volatile private var appList: Seq[ApplicationHistoryInfo] = Nil
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTimeMs()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing() = appList
    +
    +  override def getAppUI(appId: String): SparkUI = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)._2
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  override def getConfig(): Map[String, String] =
    +    Map(("Event Log Location" -> logDir))
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, by not reading
    +   * applications that haven't been updated since last time the logs were checked.
    +   */
    +  private def checkForLogs() = {
    +    lastLogCheckTimeMs = getMonotonicTimeMs()
    +    logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs))
    +    try {
    +      val logStatus = fs.listStatus(new Path(logDir))
    +      val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
    +      val logInfos = logDirs
    +        .filter {
    +            dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))
    +          }
    +
    +      val currentApps = Map[String, ApplicationHistoryInfo](
    +        appList.map(app => (app.id -> app)):_*)
    +
    +      // For any application that either (i) is not listed or (ii) has changed since the last time
    +      // the listing was created (defined by the log dir's modification time), load the app's info.
    +      // Otherwise just reuse what's already in memory.
    +      val newApps = new mutable.ArrayBuffer[ApplicationHistoryInfo](logInfos.size)
    +      for (dir <- logInfos) {
    +        val curr = currentApps.getOrElse(dir.getPath().getName(), null)
    +        if (curr == null || curr.lastUpdated < getModificationTime(dir)) {
    +          try {
    +            newApps += loadAppInfo(dir, false)._1
    +          } catch {
    +            case e: Exception => logError(s"Failed to load app info from directory $dir.")
    +          }
    +        } else {
    +          newApps += curr
    +        }
    +      }
    +
    +      appList = newApps.sortBy { info => -info.endTime }
    +    } catch {
    +      case t: Throwable => logError("Exception in checking for event log updates", t)
    +    }
    +  }
    +
    +  /**
    +   * Parse the application's logs to find out the information we need to build the
    +   * listing page.
    +   *
    +   * When creating the listing of available apps, there is no need to load the whole UI for the
    +   * application. The UI is requested by the HistoryServer (by calling getAppInfo()) when the user
    +   * clicks on a specific application.
    +   *
    +   * @param logDir Directory with application's log files.
    +   * @param renderUI Whether to create the SparkUI for the application.
    +   * @return A 2-tuple `(app info, ui)`. `ui` will be null if `renderUI` is false.
    --- End diff --
    
    The amount of duplicate code is actually minimal (3 lines), and I think it is worth having it in exchange for simpler method signatures. It's fine to keep it as is. This is not a huge deal anyway.
    
    (This is tangential to our discussion, but the reason SparkUI takes in a ListenerBus is because it must listen for events from some source. For live SparkUIs, it listens for live events issued by the DAGScheduler, and for dead SparkUIs, it listens for replayed events from logs. If SparkUI has a method called `replayLogs` or something, then it will have undefined consequences for a live SparkUI.)


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13620356
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala ---
    @@ -67,10 +69,29 @@ private[spark] class HistoryServerArguments(args: Array[String]) {
     
       private def printUsageAndExit(exitCode: Int) {
         System.err.println(
    -      "Usage: HistoryServer [options]\n" +
    -      "\n" +
    -      "Options:\n" +
    -      "  -d DIR,  --dir DIR     Location of event log files")
    +      """
    +      |Usage: HistoryServer [-d logDir]
    +      |
    +      |The preferred way to pass options is to set the configuration below using
    +      |SPARK_HISTORY_OPTS. The "-d" command line argument is avalable for backwards
    --- End diff --
    
    Then maybe I would update the usage to `<-d logDir>` to indicate that it's optional


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-46742992
  
     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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13563978
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -251,30 +172,31 @@ class HistoryServer(
     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
    -
    -  // How many applications to retain
    -  val RETAINED_APPLICATIONS = conf.getInt("spark.history.retainedApplications", 250)
    -
    -  // The port to which the web UI is bound
    -  val WEB_UI_PORT = conf.getInt("spark.history.ui.port", 18080)
    -
    -  // set whether to enable or disable view acls for all applications
    -  val HISTORY_UI_ACLS_ENABLED = conf.getBoolean("spark.history.ui.acls.enable", false)
    -
    -  val STATIC_RESOURCE_DIR = SparkUI.STATIC_RESOURCE_DIR
    -
       def main(argStrings: Array[String]) {
         initSecurity()
    -    val args = new HistoryServerArguments(argStrings)
    +    parse(argStrings.toList)
         val securityManager = new SecurityManager(conf)
    -    val server = new HistoryServer(args.logDir, securityManager, conf)
    +
    +    val providerName = conf.getOption("spark.history.provider")
    +      .getOrElse(classOf[FsHistoryProvider].getName())
    +    val provider = Class.forName(providerName)
    +      .getConstructor(classOf[SparkConf])
    +      .newInstance(conf)
    +      .asInstanceOf[ApplicationHistoryProvider]
    +
    +    val port = conf.getInt("spark.history.ui.port", 18080)
    +
    +    val server = new HistoryServer(conf, provider, securityManager, port)
         server.bind()
     
    +    Runtime.getRuntime().addShutdownHook(new Thread("HistoryServerStopper") {
    +        override def run() = {
    +          server.stop()
    +        }
    +      })
    +
         // Wait until the end of the world... or if the HistoryServer process is manually stopped
         while(true) { Thread.sleep(Int.MaxValue) }
    -    server.stop()
    --- End diff --
    
    Good catch. This was dead code and we never did actually stop the server.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-46267854
  
    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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13618945
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,210 @@
    +/*
    + * 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.io.FileNotFoundException
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval",
    +    conf.getInt("spark.history.updateInterval", 10)) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  @volatile private var appList: Seq[ApplicationHistoryInfo] = Nil
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTimeMs()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing() = appList
    +
    +  override def getAppUI(appId: String): SparkUI = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)._2
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  override def getConfig(): Map[String, String] =
    +    Map(("Event Log Location" -> logDir))
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, by not reading
    +   * applications that haven't been updated since last time the logs were checked.
    +   */
    +  private def checkForLogs() = {
    +    lastLogCheckTimeMs = getMonotonicTimeMs()
    +    logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs))
    +    try {
    +      val logStatus = fs.listStatus(new Path(logDir))
    +      val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
    +      val logInfos = logDirs
    +        .filter {
    +            dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))
    +          }
    +
    +      val currentApps = Map[String, ApplicationHistoryInfo](
    +        appList.map(app => (app.id -> app)):_*)
    +
    +      // For any application that either (i) is not listed or (ii) has changed since the last time
    +      // the listing was created (defined by the log dir's modification time), load the app's info.
    +      // Otherwise just reuse what's already in memory.
    +      val newApps = new mutable.ArrayBuffer[ApplicationHistoryInfo](logInfos.size)
    +      for (dir <- logInfos) {
    +        val curr = currentApps.getOrElse(dir.getPath().getName(), null)
    +        if (curr == null || curr.lastUpdated < getModificationTime(dir)) {
    +          try {
    +            newApps += loadAppInfo(dir, false)._1
    +          } catch {
    +            case e: Exception => logError(s"Failed to load app info from directory $dir.")
    +          }
    +        } else {
    +          newApps += curr
    +        }
    +      }
    +
    +      appList = newApps.sortBy { info => -info.endTime }
    +    } catch {
    +      case t: Throwable => logError("Exception in checking for event log updates", t)
    +    }
    +  }
    +
    +  /**
    +   * Parse the application's logs to find out the information we need to build the
    +   * listing page.
    +   *
    +   * When creating the listing of available apps, there is no need to load the whole UI for the
    +   * application. The UI is requested by the HistoryServer (by calling getAppInfo()) when the user
    +   * clicks on a specific application.
    +   *
    +   * @param logDir Directory with application's log files.
    +   * @param renderUI Whether to create the SparkUI for the application.
    +   * @return A 2-tuple `(app info, ui)`. `ui` will be null if `renderUI` is false.
    --- End diff --
    
    This is a private method, so I'm not to worried about the ugly interface. My main goal here is to share the code to set up and drive the replay bus. The UI makes it really hard to break this up, because it takes the replay bus as a constructor argument, so you can't just have something like a "replayLog(logFile, SparkListener*)" and call that with a SparkUI object.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13620076
  
    --- Diff: docs/monitoring.md ---
    @@ -69,7 +69,7 @@ represents an application's event logs. This creates a web interface at
     <table class="table">
       <tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
       <tr>
    -    <td>spark.history.updateInterval</td>
    +    <td>spark.history.fs.updateInterval</td>
    --- End diff --
    
    You also need to document the new configs you added (e.g. `spark.history.provider`)


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-44446957
  
    Update is just a rebase (with just minor cleanups).


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13665669
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,210 @@
    +/*
    + * 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.io.FileNotFoundException
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval",
    +    conf.getInt("spark.history.updateInterval", 10)) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  @volatile private var appList: Seq[ApplicationHistoryInfo] = Nil
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTimeMs()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing() = appList
    +
    +  override def getAppUI(appId: String): SparkUI = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)._2
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  override def getConfig(): Map[String, String] =
    +    Map(("Event Log Location" -> logDir))
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, by not reading
    +   * applications that haven't been updated since last time the logs were checked.
    +   */
    +  private def checkForLogs() = {
    +    lastLogCheckTimeMs = getMonotonicTimeMs()
    +    logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs))
    +    try {
    +      val logStatus = fs.listStatus(new Path(logDir))
    +      val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
    +      val logInfos = logDirs
    +        .filter {
    +            dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))
    +          }
    +
    +      val currentApps = Map[String, ApplicationHistoryInfo](
    +        appList.map(app => (app.id -> app)):_*)
    +
    +      // For any application that either (i) is not listed or (ii) has changed since the last time
    +      // the listing was created (defined by the log dir's modification time), load the app's info.
    +      // Otherwise just reuse what's already in memory.
    +      val newApps = new mutable.ArrayBuffer[ApplicationHistoryInfo](logInfos.size)
    +      for (dir <- logInfos) {
    +        val curr = currentApps.getOrElse(dir.getPath().getName(), null)
    +        if (curr == null || curr.lastUpdated < getModificationTime(dir)) {
    +          try {
    +            newApps += loadAppInfo(dir, false)._1
    +          } catch {
    +            case e: Exception => logError(s"Failed to load app info from directory $dir.")
    +          }
    +        } else {
    +          newApps += curr
    +        }
    +      }
    +
    +      appList = newApps.sortBy { info => -info.endTime }
    +    } catch {
    +      case t: Throwable => logError("Exception in checking for event log updates", t)
    +    }
    +  }
    +
    +  /**
    +   * Parse the application's logs to find out the information we need to build the
    +   * listing page.
    +   *
    +   * When creating the listing of available apps, there is no need to load the whole UI for the
    +   * application. The UI is requested by the HistoryServer (by calling getAppInfo()) when the user
    +   * clicks on a specific application.
    +   *
    +   * @param logDir Directory with application's log files.
    +   * @param renderUI Whether to create the SparkUI for the application.
    +   * @return A 2-tuple `(app info, ui)`. `ui` will be null if `renderUI` is false.
    --- End diff --
    
    Also remember that the method that would generate the UI has to contain pretty much all the code from the method that generates just the listing info, since it needs to get the view acls. So while it may be "just a few lines", it's the concept of duplicating all this knowledge that bothers me.
    
    As for the SparkUI, I was suggesting that the class itself should be a SparkListener, and then its construction would be decoupled from the bus. Internally it can fan out events to whatever listeners do the actual work, but that would keep its interface cleaner.
    
    (Or optionally have some sort of "registerListener(SparkListenerBus)" method, as long as the bus is not passed to the constructor.)


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13398616
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -290,8 +217,88 @@ object HistoryServer {
         }
       }
     
    +  private def parse(args: List[String]): Unit = {
    +    args match {
    +      case ("--dir" | "-d") :: value :: tail =>
    +        set("fs.logDirectory",  value)
    +        parse(tail)
    +
    +      case ("--port" | "-p") :: value :: tail =>
    +        set("ui.port", value)
    +        parse(tail)
    +
    +      case ("-D") :: opt :: value :: tail =>
    --- End diff --
    
    See previous comment about config file vs. command line options, regarding SPARK_HISTORY_OPTS. I'm not a big fan of that approach for daemon processes - config files are a cleaner approach here.
    
    I can drop the "-D" since it's mostly just a convenience (avoids having to type "spark.history." before every option), or I can change its syntax to be the more "standard" -Dkey=value. Your choice.
    
    As far as backwards compat, I tried to keep the 1.0 options intact. I'll double check it 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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-46741637
  
     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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13620284
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala ---
    @@ -67,10 +69,29 @@ private[spark] class HistoryServerArguments(args: Array[String]) {
     
       private def printUsageAndExit(exitCode: Int) {
         System.err.println(
    -      "Usage: HistoryServer [options]\n" +
    -      "\n" +
    -      "Options:\n" +
    -      "  -d DIR,  --dir DIR     Location of event log files")
    +      """
    +      |Usage: HistoryServer [-d logDir]
    +      |
    +      |The preferred way to pass options is to set the configuration below using
    +      |SPARK_HISTORY_OPTS. The "-d" command line argument is avalable for backwards
    --- End diff --
    
    Not sure if we want to say this, actually. Right now the `start-history-server.sh` script still expects a logging directory, which uses the now "deprecated" way as of this PR. I would either remove this comment or change the way we use the script.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-45041212
  
    Hey @vanzin - I just did a pass on this. From what I can tell this adds four somewhat independent changes:
    
    1. Caching applications that we know haven't changed.
    2. Adding pagination to the list of applications.
    3. Abstracting the history provider. 
    4. Replacing the way arguments are specified to the server.
    
    Of these, I feel that (1) and (2) are great improvements. It might be nice to pull those out into a separate PR, because here they are somewhat conflated with (3) and (4).
    
    For (3) - I think it might make more sense to wait until we have two concrete implementations here before we decided to make an interface. Otherwise I think it's very likely we'd end up changing the interface anyways down the road. And this isn't a public API - we can easily just generalize it whenever we want. @andrewor14 can comment on this more next week when he's back, since he made this original design.
    
    For (4) the proposal here, I wasn't sure exactly what problem it was trying to solve. It would be good to have more information on 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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-45557107
  
    @tgraves Can you take a look at whether the semantics of the history server ACLs are preserved? I took a pass over it and it seems reasonable to me, but I could have totally missed some things on the way.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13570773
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,193 @@
    +/*
    + * 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.io.FileNotFoundException
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval", 10) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  @volatile private var appList: Seq[ApplicationHistoryInfo] = Nil
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTime()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing() = appList
    +
    +  override def getAppInfo(appId: String): ApplicationHistoryInfo = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, by not reading
    +   * applications that haven't been updated since last time the logs were checked.
    +   */
    +  def checkForLogs() = {
    +    lastLogCheckTimeMs = getMonotonicTime()
    +    logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs))
    +    try {
    +      val logStatus = fs.listStatus(new Path(logDir))
    +      val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
    +      val logInfos = logDirs
    +        .filter {
    +            dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))
    +          }
    +
    +      val currentApps = Map[String, ApplicationHistoryInfo](
    +        appList.map(app => (app.id -> app)):_*)
    +
    +      // For any application that either (i) is not listed or (ii) has changed since the last time
    +      // the listing was created (defined by the log dir's modification time), load the app's info.
    +      // Otherwise just reuse what's already in memory.
    +      val newApps = new mutable.ArrayBuffer[ApplicationHistoryInfo](logInfos.size)
    +      for (dir <- logInfos) {
    +        val curr = currentApps.getOrElse(dir.getPath().getName(), null)
    +        if (curr == null || curr.lastUpdated < getModificationTime(dir)) {
    +          try {
    +            newApps += loadAppInfo(dir, false)
    --- End diff --
    
    Are you asking specifically about the "false" argument? If so, yes; at this point we don't want to create the UI for the application, to avoid loading every application's UI in memory (and consequently keeping memory usage under control).


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13564425
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -290,8 +212,81 @@ object HistoryServer {
         }
       }
     
    +  private def parse(args: List[String]): Unit = {
    +    args match {
    +      case ("--dir" | "-d") :: value :: tail =>
    +        set("fs.logDirectory",  value)
    +        parse(tail)
    +
    +      case ("-D") :: opt :: value :: tail =>
    +        set(opt, value)
    +        parse(tail)
    +
    +      case ("--help" | "-h") :: tail =>
    +        printUsageAndExit(0)
    +
    +      case Nil =>
    +
    +      case _ =>
    +        printUsageAndExit(1)
    +    }
    +  }
    +
    +  private def set(name: String, value: String) = {
    +    conf.set("spark.history." + name, value)
    +  }
    +
    +  private def printUsageAndExit(exitCode: Int) {
    +    System.err.println(
    +      """
    +      |Usage: HistoryServer [options]
    +      |
    +      |Options are set by passing "-D option value" command line arguments to the class.
    +      |Command line options will override JVM system properties (which should be prepended
    +      |with "spark.history.").
    +      |
    +      |History Server options are always available; additional options depend on the provider.
    +      |
    +      |History Server options:
    +      |
    +      |  ui.port           Port where server will listen for connections (default 18080)
    +      |  ui.acls.enable    Whether to enable view acls for all applications (default false)
    +      |  provider          Name of history provider class (defaults to file system-based provider)
    +      |
    +      |FsHistoryProvider options:
    +      |
    +      |  fs.logDirectory   Directory where app logs are stored (required)
    +      |  fs.updateInterval How often to reload log data from storage (seconds, default 10)
    +      |""".stripMargin)
    +    System.exit(exitCode)
    +  }
    +
     }
     
    +private[spark] abstract class ApplicationHistoryProvider {
    --- End diff --
    
    This technically doesn't belong to the HistoryServer. I would move this to its own file.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13667055
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,210 @@
    +/*
    + * 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.io.FileNotFoundException
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval",
    +    conf.getInt("spark.history.updateInterval", 10)) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  @volatile private var appList: Seq[ApplicationHistoryInfo] = Nil
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTimeMs()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing() = appList
    +
    +  override def getAppUI(appId: String): SparkUI = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)._2
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  override def getConfig(): Map[String, String] =
    +    Map(("Event Log Location" -> logDir))
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, by not reading
    +   * applications that haven't been updated since last time the logs were checked.
    +   */
    +  private def checkForLogs() = {
    +    lastLogCheckTimeMs = getMonotonicTimeMs()
    +    logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs))
    +    try {
    +      val logStatus = fs.listStatus(new Path(logDir))
    +      val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
    +      val logInfos = logDirs
    +        .filter {
    +            dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))
    +          }
    +
    +      val currentApps = Map[String, ApplicationHistoryInfo](
    +        appList.map(app => (app.id -> app)):_*)
    +
    +      // For any application that either (i) is not listed or (ii) has changed since the last time
    +      // the listing was created (defined by the log dir's modification time), load the app's info.
    +      // Otherwise just reuse what's already in memory.
    +      val newApps = new mutable.ArrayBuffer[ApplicationHistoryInfo](logInfos.size)
    +      for (dir <- logInfos) {
    +        val curr = currentApps.getOrElse(dir.getPath().getName(), null)
    +        if (curr == null || curr.lastUpdated < getModificationTime(dir)) {
    +          try {
    +            newApps += loadAppInfo(dir, false)._1
    +          } catch {
    +            case e: Exception => logError(s"Failed to load app info from directory $dir.")
    +          }
    +        } else {
    +          newApps += curr
    +        }
    +      }
    +
    +      appList = newApps.sortBy { info => -info.endTime }
    +    } catch {
    +      case t: Throwable => logError("Exception in checking for event log updates", t)
    +    }
    +  }
    +
    +  /**
    +   * Parse the application's logs to find out the information we need to build the
    +   * listing page.
    +   *
    +   * When creating the listing of available apps, there is no need to load the whole UI for the
    +   * application. The UI is requested by the HistoryServer (by calling getAppInfo()) when the user
    +   * clicks on a specific application.
    +   *
    +   * @param logDir Directory with application's log files.
    +   * @param renderUI Whether to create the SparkUI for the application.
    +   * @return A 2-tuple `(app info, ui)`. `ui` will be null if `renderUI` is false.
    --- End diff --
    
    Yes, you're right (a `registerListener` approach has the same problem as the constructor).


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-46768285
  
    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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13630883
  
    --- Diff: docs/monitoring.md ---
    @@ -35,11 +35,13 @@ If Spark is run on Mesos or YARN, it is still possible to reconstruct the UI of
     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>
    +    ./sbin/start-history-server.sh
     
    -The base logging directory must be supplied, and should contain sub-directories that each
    -represents an application's event logs. This creates a web interface at
    -`http://<server-url>:18080` by default. The history server can be configured as follows:
    +When using the file-system provider class (see spark.history.provider below), the base logging
    +directory must be supplied in the "spark.history.fs.logDirectory" configuration option, and should
    --- End diff --
    
    minor: use `<code></code>` around the option name


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13564378
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -290,8 +212,81 @@ object HistoryServer {
         }
       }
     
    +  private def parse(args: List[String]): Unit = {
    +    args match {
    +      case ("--dir" | "-d") :: value :: tail =>
    +        set("fs.logDirectory",  value)
    +        parse(tail)
    +
    +      case ("-D") :: opt :: value :: tail =>
    +        set(opt, value)
    +        parse(tail)
    +
    +      case ("--help" | "-h") :: tail =>
    +        printUsageAndExit(0)
    +
    +      case Nil =>
    +
    +      case _ =>
    +        printUsageAndExit(1)
    +    }
    +  }
    +
    +  private def set(name: String, value: String) = {
    +    conf.set("spark.history." + name, value)
    --- End diff --
    
    I find this confusing. In other places we require the user to specify the full config name. We should try to keep the configuration of the HistoryServer 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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13619455
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,210 @@
    +/*
    + * 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.io.FileNotFoundException
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval",
    +    conf.getInt("spark.history.updateInterval", 10)) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  @volatile private var appList: Seq[ApplicationHistoryInfo] = Nil
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTimeMs()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing() = appList
    +
    +  override def getAppUI(appId: String): SparkUI = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)._2
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  override def getConfig(): Map[String, String] =
    +    Map(("Event Log Location" -> logDir))
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, by not reading
    +   * applications that haven't been updated since last time the logs were checked.
    +   */
    +  private def checkForLogs() = {
    +    lastLogCheckTimeMs = getMonotonicTimeMs()
    +    logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs))
    +    try {
    +      val logStatus = fs.listStatus(new Path(logDir))
    +      val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
    +      val logInfos = logDirs
    +        .filter {
    +            dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))
    +          }
    +
    +      val currentApps = Map[String, ApplicationHistoryInfo](
    +        appList.map(app => (app.id -> app)):_*)
    +
    +      // For any application that either (i) is not listed or (ii) has changed since the last time
    +      // the listing was created (defined by the log dir's modification time), load the app's info.
    +      // Otherwise just reuse what's already in memory.
    +      val newApps = new mutable.ArrayBuffer[ApplicationHistoryInfo](logInfos.size)
    +      for (dir <- logInfos) {
    +        val curr = currentApps.getOrElse(dir.getPath().getName(), null)
    +        if (curr == null || curr.lastUpdated < getModificationTime(dir)) {
    +          try {
    +            newApps += loadAppInfo(dir, false)._1
    +          } catch {
    +            case e: Exception => logError(s"Failed to load app info from directory $dir.")
    +          }
    +        } else {
    +          newApps += curr
    +        }
    +      }
    +
    +      appList = newApps.sortBy { info => -info.endTime }
    +    } catch {
    +      case t: Throwable => logError("Exception in checking for event log updates", t)
    +    }
    +  }
    +
    +  /**
    +   * Parse the application's logs to find out the information we need to build the
    +   * listing page.
    +   *
    +   * When creating the listing of available apps, there is no need to load the whole UI for the
    +   * application. The UI is requested by the HistoryServer (by calling getAppInfo()) when the user
    +   * clicks on a specific application.
    +   *
    +   * @param logDir Directory with application's log files.
    +   * @param renderUI Whether to create the SparkUI for the application.
    +   * @return A 2-tuple `(app info, ui)`. `ui` will be null if `renderUI` is false.
    --- End diff --
    
    I dislike the duplication of code to parse logs / create the replay listener / drive it caused by your code. Note that in both cases I need the ApplicationEventListener, since that's where the ACLs come from.
    
    I can move the ACL set up to getAppUI() to avoid the null check, but otherwise, I rather prefer the current code.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13565262
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -38,56 +39,75 @@ import org.apache.spark.util.Utils
      * 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,
    +    provider: ApplicationHistoryProvider,
         securityManager: SecurityManager,
    -    conf: SparkConf)
    -  extends WebUI(securityManager, HistoryServer.WEB_UI_PORT, conf) with Logging {
    +    port: Int)
    +  extends WebUI(securityManager, port, conf) with Logging {
    +
    +  // How many applications to retain
    +  private val retainedApplications = conf.getInt("spark.history.retainedApplications", 50)
     
    -  import HistoryServer._
    +  // set whether to enable or disable view acls for all applications
    +  private val uiAclsEnabled = conf.getBoolean("spark.history.ui.acls.enable", false)
     
    -  private val fileSystem = Utils.getHadoopFileSystem(baseLogDir)
       private val localHost = Utils.localHostName()
       private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHost)
     
    -  // A timestamp of when the disk was last accessed to check for log updates
    -  private var lastLogCheckTime = -1L
    +  private val appLoader = new CacheLoader[String, SparkUI] {
    +    override def load(key: String): SparkUI = {
    +      val info = provider.getAppInfo(key)
    +      if (info == null) {
    +        throw new NoSuchElementException()
    +      }
    +      info.ui.getSecurityManager.setUIAcls(uiAclsEnabled)
    +      info.ui.getSecurityManager.setViewAcls(info.sparkUser, info.viewAcls)
    +      attachSparkUI(info.ui)
    +      info.ui
    +    }
    +  }
     
    -  // Number of completed applications found in this directory
    -  private var numCompletedApplications = 0
    +  private val appCache = CacheBuilder.newBuilder()
    +    .maximumSize(retainedApplications)
    +    .removalListener(new RemovalListener[String, SparkUI] {
    +      override def onRemoval(rm: RemovalNotification[String, SparkUI]) = {
    +        detachSparkUI(rm.getValue())
    +      }
    +    })
    +    .build(appLoader)
    +
    +  private val loaderServlet = new HttpServlet {
    +    protected override def doGet(req: HttpServletRequest, res: HttpServletResponse): Unit = {
    +      val parts = req.getPathInfo().split("/")
    --- End diff --
    
    We should do a null check here. According to the docs, `getPathInfo` returns null if there is no extra path information. Although we only invoke this servlet through `/history/*` at the moment, it would be good to keep this general so we don't run into random NPEs if we decide to change the path.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13401584
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -290,8 +217,88 @@ object HistoryServer {
         }
       }
     
    +  private def parse(args: List[String]): Unit = {
    +    args match {
    +      case ("--dir" | "-d") :: value :: tail =>
    +        set("fs.logDirectory",  value)
    +        parse(tail)
    +
    +      case ("--port" | "-p") :: value :: tail =>
    +        set("ui.port", value)
    +        parse(tail)
    +
    +      case ("-D") :: opt :: value :: tail =>
    +        set(opt, value)
    +        parse(tail)
    +
    +      case ("--help" | "-h") :: tail =>
    +        printUsageAndExit(0)
    +
    +      case Nil =>
    +
    +      case _ =>
    +        printUsageAndExit(1)
    +    }
    +  }
    +
    +  private def set(name: String, value: String) = {
    +    conf.set("spark.history." + name, value)
    +  }
    +
    +  private def printUsageAndExit(exitCode: Int) {
    +    System.err.println(
    +      """
    +      |Usage: HistoryServer [options]
    +      |
    +      |Options are set by passing "-D option value" command line arguments to the class.
    +      |Command line options will override the Spark configuration file and system properties.
    +      |History Server options are always available; additional options depend on the provider.
    +      |
    +      |History Server options:
    +      |
    +      |  ui.port           Port where server will listen for connections (default 18080)
    +      |  ui.acls.enable    Whether to enable view acls for all applications (default false)
    +      |  provider          Name of history provider class (defaults to file system-based provider)
    +      |
    +      |FsHistoryProvider options:
    +      |
    +      |  fs.logDirectory   Directory where app logs are stored (required)
    +      |  fs.updateInterval How often to reload log data from storage (seconds, default 10)
    +      |""".stripMargin)
    +    System.exit(exitCode)
    +  }
    +
     }
     
    +private[spark] abstract class ApplicationHistoryProvider {
    +
    +  /**
    +   * This method should return a list of applications available for the history server to
    +   * show. The listing is assumed to be in descending time order.
    +   *
    +   * An adjusted offset should be returned if the app list has changed and the request
    +   * references an invalid start offset. Otherwise, the provided offset should be returned.
    +   *
    +   * @param offset Starting offset for returned objects.
    +   * @param count Max number of objects to return.
    +   * @return 3-tuple (requested app list, adjusted offset, count of all available apps)
    +   */
    +  def getListing(offset: Int, count: Int): (Seq[ApplicationHistoryInfo], Int, Int)
    --- End diff --
    
    re: two methods, that's exactly what my change does. getListing() returns the listing only, and getAppInfo() returns the information to render the UI. (I'm open to better names if that's what confused you.) 
    
    The point of having offset in the listing API is to optimize the listing itself. We're only showing a handful of items to the user at a time; if the backend has, let's say, 1 million entries, removing the offset from the interface would require materializing those 1 million entries, just for them to be immediately trimmed by the UI layer. That's absurdly wasteful.
    
    The ordering is described in the getListing() method's doc header.
    
    offset / count is how any API like this works; I really would like to better understand what is really worrying you about this change.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-46376898
  
    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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13630434
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,210 @@
    +/*
    + * 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.io.FileNotFoundException
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval",
    +    conf.getInt("spark.history.updateInterval", 10)) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  @volatile private var appList: Seq[ApplicationHistoryInfo] = Nil
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTimeMs()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing() = appList
    +
    +  override def getAppUI(appId: String): SparkUI = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)._2
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  override def getConfig(): Map[String, String] =
    +    Map(("Event Log Location" -> logDir))
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, by not reading
    +   * applications that haven't been updated since last time the logs were checked.
    +   */
    +  private def checkForLogs() = {
    +    lastLogCheckTimeMs = getMonotonicTimeMs()
    +    logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs))
    +    try {
    +      val logStatus = fs.listStatus(new Path(logDir))
    +      val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
    +      val logInfos = logDirs
    +        .filter {
    +            dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))
    +          }
    +
    +      val currentApps = Map[String, ApplicationHistoryInfo](
    +        appList.map(app => (app.id -> app)):_*)
    +
    +      // For any application that either (i) is not listed or (ii) has changed since the last time
    +      // the listing was created (defined by the log dir's modification time), load the app's info.
    +      // Otherwise just reuse what's already in memory.
    +      val newApps = new mutable.ArrayBuffer[ApplicationHistoryInfo](logInfos.size)
    +      for (dir <- logInfos) {
    +        val curr = currentApps.getOrElse(dir.getPath().getName(), null)
    +        if (curr == null || curr.lastUpdated < getModificationTime(dir)) {
    +          try {
    +            newApps += loadAppInfo(dir, false)._1
    +          } catch {
    +            case e: Exception => logError(s"Failed to load app info from directory $dir.")
    +          }
    +        } else {
    +          newApps += curr
    +        }
    +      }
    +
    +      appList = newApps.sortBy { info => -info.endTime }
    +    } catch {
    +      case t: Throwable => logError("Exception in checking for event log updates", t)
    +    }
    +  }
    +
    +  /**
    +   * Parse the application's logs to find out the information we need to build the
    +   * listing page.
    +   *
    +   * When creating the listing of available apps, there is no need to load the whole UI for the
    +   * application. The UI is requested by the HistoryServer (by calling getAppInfo()) when the user
    +   * clicks on a specific application.
    +   *
    +   * @param logDir Directory with application's log files.
    +   * @param renderUI Whether to create the SparkUI for the application.
    +   * @return A 2-tuple `(app info, ui)`. `ui` will be null if `renderUI` is false.
    --- End diff --
    
    I see, we still get the view ACLs for rendering the UI.
    
    I think the ugliness of returning a tuple should be avoided, even if it is not user facing. These are really two separate cases anyway, so the correct way to do this is to abstract out the duplicate part (i.e. the instantiation of the ReplayBus), and then have both `getAppUI` and `getAppInfo` use the same method. Right now, based on the method signature, it seems that the caller needs both items of the tuple, which is not true.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13566089
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,193 @@
    +/*
    + * 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.io.FileNotFoundException
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval", 10) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  @volatile private var appList: Seq[ApplicationHistoryInfo] = Nil
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTime()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing() = appList
    +
    +  override def getAppInfo(appId: String): ApplicationHistoryInfo = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, by not reading
    +   * applications that haven't been updated since last time the logs were checked.
    +   */
    +  def checkForLogs() = {
    +    lastLogCheckTimeMs = getMonotonicTime()
    +    logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs))
    +    try {
    +      val logStatus = fs.listStatus(new Path(logDir))
    +      val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
    +      val logInfos = logDirs
    +        .filter {
    +            dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))
    +          }
    +
    +      val currentApps = Map[String, ApplicationHistoryInfo](
    +        appList.map(app => (app.id -> app)):_*)
    +
    +      // For any application that either (i) is not listed or (ii) has changed since the last time
    +      // the listing was created (defined by the log dir's modification time), load the app's info.
    +      // Otherwise just reuse what's already in memory.
    +      val newApps = new mutable.ArrayBuffer[ApplicationHistoryInfo](logInfos.size)
    +      for (dir <- logInfos) {
    +        val curr = currentApps.getOrElse(dir.getPath().getName(), null)
    +        if (curr == null || curr.lastUpdated < getModificationTime(dir)) {
    +          try {
    +            newApps += loadAppInfo(dir, false)
    --- End diff --
    
    Just so I understand, is the purpose of this just to list all applications, without rendering them, but only if an HTTP request comes in do we render 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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-46267638
  
    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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-46900448
  
    Thanks @vanzin for this feature. I'm going to merge 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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-46347026
  
    Sorry about the noise, I was testing some local script and wrongly messed up this branch.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13566249
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,203 @@
    +/*
    + * 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.io.FileNotFoundException
    +import java.util.concurrent.atomic.AtomicReference
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval", 10) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  private val appList = new AtomicReference[Seq[ApplicationHistoryInfo]](Nil)
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTime()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing(offset: Int, count: Int) = {
    +    val list = appList.get()
    +    val theOffset = if (offset < list.size) offset else 0
    +    (list.slice(theOffset, Math.min(theOffset + count, list.size)), theOffset, list.size)
    +  }
    +
    +  override def getAppInfo(appId: String): ApplicationHistoryInfo = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, but not reading
    +   * applications that hasn't been updated since last time the logs were checked.
    +   */
    +  def checkForLogs() = synchronized {
    +    lastLogCheckTimeMs = getMonotonicTime()
    +    logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs))
    +    try {
    +      val logStatus = fs.listStatus(new Path(logDir))
    +      val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
    +      val logInfos = logDirs
    +        .filter {
    +            dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))
    +          }
    +
    +      var currentApps = Map[String, ApplicationHistoryInfo](
    +        appList.get().map(app => (app.id -> app)):_*)
    +
    +      // For any application that either (i) is not listed or (ii) has changed since the last time
    +      // the listing was created (defined by the log dir's modification time), load the app's info.
    +      // Otherwise just reuse what's already in memory.
    +      val newApps = new mutable.ArrayBuffer[ApplicationHistoryInfo](logInfos.size)
    +      for (dir <- logInfos) {
    +        val curr = currentApps.getOrElse(dir.getPath().getName(), null)
    +        if (curr == null || curr.lastUpdated < getModificationTime(dir)) {
    +          try {
    +            newApps += loadAppInfo(dir, false)
    +          } catch {
    +            case e: Exception => logError(s"Failed to load app info from directory $dir.")
    +          }
    +        } else {
    +          newApps += curr
    +        }
    +      }
    +
    +      appList.set(newApps.sortBy { info => -info.lastUpdated })
    +    } catch {
    +      case t: Throwable => logError("Exception in checking for event log updates", t)
    +    }
    +  }
    +
    +  /**
    +   * Parse the application's logs to find out the information we need to build the
    +   * listing page.
    +   */
    +  private def loadAppInfo(logDir: FileStatus, renderUI: Boolean): ApplicationHistoryInfo = {
    +    val elogInfo = EventLoggingListener.parseLoggingInfo(logDir.getPath(), fs)
    +    val path = logDir.getPath
    +    val appId = path.getName
    +    val replayBus = new ReplayListenerBus(elogInfo.logPaths, fs, elogInfo.compressionCodec)
    +    val appListener = new ApplicationEventListener
    +    replayBus.addListener(appListener)
    +
    +    val ui: SparkUI = if (renderUI) {
    +        val conf = this.conf.clone()
    +        val appSecManager = new SecurityManager(conf)
    +        new SparkUI(conf, appSecManager, replayBus, appId, "/history/" + appId)
    +        // Do not call ui.bind() to avoid creating a new server for each application
    +      } else {
    +        null
    +      }
    +
    +    replayBus.replay()
    +    val appName = appListener.appName
    +    val sparkUser = appListener.sparkUser
    +    val startTime = appListener.startTime
    +    val endTime = appListener.endTime
    +    val lastUpdated = getModificationTime(logDir)
    +    ApplicationHistoryInfo(appId,
    +      appListener.appName,
    +      appListener.startTime,
    +      appListener.endTime,
    +      getModificationTime(logDir),
    +      appListener.sparkUser,
    +      if (renderUI) appListener.viewAcls else null,
    +      ui)
    +  }
    +
    +  /** Return when this directory was last modified. */
    +  private def getModificationTime(dir: FileStatus): Long = {
    +    try {
    +      val logFiles = fs.listStatus(dir.getPath)
    +      if (logFiles != null && !logFiles.isEmpty) {
    +        logFiles.map(_.getModificationTime).max
    +      } else {
    +        dir.getModificationTime
    +      }
    +    } catch {
    +      case t: Throwable =>
    +        logError("Exception in accessing modification time of %s".format(dir.getPath), t)
    +        -1L
    +    }
    +  }
    +
    +  /** Returns the system's mononotically increasing time. */
    +  private def getMonotonicTime() = System.nanoTime() / (1000 * 1000)
    --- End diff --
    
    Might be good to say what unit this returns, e.g. `getMonotonicTimeMs`


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13564491
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -290,8 +212,81 @@ object HistoryServer {
         }
       }
     
    +  private def parse(args: List[String]): Unit = {
    +    args match {
    +      case ("--dir" | "-d") :: value :: tail =>
    +        set("fs.logDirectory",  value)
    +        parse(tail)
    +
    +      case ("-D") :: opt :: value :: tail =>
    +        set(opt, value)
    +        parse(tail)
    +
    +      case ("--help" | "-h") :: tail =>
    +        printUsageAndExit(0)
    +
    +      case Nil =>
    +
    +      case _ =>
    +        printUsageAndExit(1)
    +    }
    +  }
    +
    +  private def set(name: String, value: String) = {
    +    conf.set("spark.history." + name, value)
    +  }
    +
    +  private def printUsageAndExit(exitCode: Int) {
    +    System.err.println(
    +      """
    +      |Usage: HistoryServer [options]
    +      |
    +      |Options are set by passing "-D option value" command line arguments to the class.
    +      |Command line options will override JVM system properties (which should be prepended
    +      |with "spark.history.").
    +      |
    +      |History Server options are always available; additional options depend on the provider.
    +      |
    +      |History Server options:
    +      |
    +      |  ui.port           Port where server will listen for connections (default 18080)
    +      |  ui.acls.enable    Whether to enable view acls for all applications (default false)
    +      |  provider          Name of history provider class (defaults to file system-based provider)
    +      |
    +      |FsHistoryProvider options:
    +      |
    +      |  fs.logDirectory   Directory where app logs are stored (required)
    +      |  fs.updateInterval How often to reload log data from storage (seconds, default 10)
    --- End diff --
    
    You should also update the documentation in `configuration.md`


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-45128194
  
    Okay - we can keep the changes together. I think the test-ability argument is a good one for abstracting away the interface. We typically try not to add interfaces until we have two concrete implementations... but again, for internal code it's not a big deal since we can always refactor it later.
    
    Re: reviewboard, we have a bunch of project automation around github, but it's possible reveiwboard could augment this somehow (not sure). Would be interesting to look into. There is no way to do an incremental diff in github, which is indeed super annoying, however in the past that feature hasn't been seen as outweighing the other benefits. I'd guess people are open to experimentation!


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13366096
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,203 @@
    +/*
    + * 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.io.FileNotFoundException
    +import java.util.concurrent.atomic.AtomicReference
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval", 10) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  private val appList = new AtomicReference[Seq[ApplicationHistoryInfo]](Nil)
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTime()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing(offset: Int, count: Int) = {
    +    val list = appList.get()
    +    val theOffset = if (offset < list.size) offset else 0
    +    (list.slice(theOffset, Math.min(theOffset + count, list.size)), theOffset, list.size)
    +  }
    +
    +  override def getAppInfo(appId: String): ApplicationHistoryInfo = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, but not reading
    +   * applications that hasn't been updated since last time the logs were checked.
    +   */
    +  def checkForLogs() = synchronized {
    --- End diff --
    
    Does this need to be synchronzied given the new re-factoring? What if you just added a lock around the `appList` (as discussed earlier) would that be sufficient?


---
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-1768] History server enhancements.

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

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


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13615472
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,193 @@
    +/*
    + * 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.io.FileNotFoundException
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval", 10) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  @volatile private var appList: Seq[ApplicationHistoryInfo] = Nil
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTime()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing() = appList
    +
    +  override def getAppInfo(appId: String): ApplicationHistoryInfo = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, by not reading
    +   * applications that haven't been updated since last time the logs were checked.
    +   */
    +  def checkForLogs() = {
    +    lastLogCheckTimeMs = getMonotonicTime()
    +    logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs))
    +    try {
    +      val logStatus = fs.listStatus(new Path(logDir))
    +      val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
    +      val logInfos = logDirs
    +        .filter {
    +            dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))
    +          }
    +
    +      val currentApps = Map[String, ApplicationHistoryInfo](
    +        appList.map(app => (app.id -> app)):_*)
    --- End diff --
    
    I don't think there's a good solution here without a bigger change. The issue is that we want to list apps in some sensible order (current code uses end time), but when checking whether the app is up-to-date we need to compare its id. And just using a map doesn't allow us to have both.
    
    So to avoid using more memory we'd have to reload every app from HDFS when rescanning, which can be pretty slow (and, with the check interval being 10s by default, pretty unfriendly to the NN).
    
    At this point I'd like to defer a solution to this particular problem to a separate change. I've been playing with some ideas to solve this and other issues in the code, but that part is not yet ready for 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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-46742969
  
    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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-46741535
  
    I think I fixed it. 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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13572946
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala ---
    @@ -25,20 +25,32 @@ import org.apache.spark.ui.{WebUIPage, UIUtils}
     
     private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") {
     
    +  val pageSize = 20
    +
       def render(request: HttpServletRequest): Seq[Node] = {
    -    val appRows = parent.appIdToInfo.values.toSeq.sortBy { app => -app.lastUpdated }
    -    val appTable = UIUtils.listingTable(appHeader, appRow, appRows)
    +    val requestedPage = Option(request.getParameter("page")).getOrElse("1").toInt
    +    val requestedFirst = (requestedPage - 1) * pageSize
    +
    +    val allApps = parent.getApplicationList()
    +    val actualFirst = if (requestedFirst < allApps.size) requestedFirst else 0
    +    val apps = allApps.slice(actualFirst, Math.min(pageSize, allApps.size))
    +
    +    val actualPage = (actualFirst / pageSize) + 1
    +    val last = Math.min(actualFirst + pageSize, allApps.size) - 1
    +    val pageCount = allApps.size / pageSize + (if (allApps.size % pageSize > 0) 1 else 0)
    +
    +    val appTable = UIUtils.listingTable(appHeader, appRow, apps)
         val content =
           <div class="row-fluid">
             <div class="span12">
    -          <ul class="unstyled">
    -            <li><strong>Event Log Location: </strong> {parent.baseLogDir}</li>
    --- End diff --
    
    Ah I see. In the future it might make sense to have a table that shows what all the configs are set to, and maybe we can make the logging directory one of these 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-1768] History server enhancements.

Posted by vanzin <gi...@git.apache.org>.
GitHub user vanzin reopened a pull request:

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

    [SPARK-1768] History server enhancements.

    Two improvements to the history server:
    
    - Separate the HTTP handling from history fetching, so that it's easy to add
      new backends later (thinking about SPARK-1537 in the long run)
    
    - Avoid loading all UIs in memory. Do lazy loading instead, keeping a few in
      memory for faster access. This allows the app limit to go away, since holding
      just the listing in memory shouldn't be too expensive unless the user has millions
      of completed apps in the history (at which point I'd expect other issues to arise
      aside from history server memory usage, such as FileSystem.listStatus()
      starting to become ridiculously expensive).
    
    I also fixed a few minor things along the way which aren't really worth mentioning.
    I also removed the app's log path from the UI since that information may not even
    exist depending on which backend is used (even though there is only one now).

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

    $ git pull https://github.com/vanzin/spark hist-server

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

    https://github.com/apache/spark/pull/718.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 #718
    
----
commit b28447862b515a45a6b7798b256014df23b55799
Author: Marcelo Vanzin <va...@cloudera.com>
Date:   2014-05-07T23:30:02Z

    Separate history server from history backend.
    
    This change does two things, mainly:
    
    - Separate the logic of serving application history from fetching
      application history from the underlying storage. Not only this
      cleans up the code a little bit, but it also serves as initial
      work for SPARK-1537, where we may want to fetch application data
      from Yarn instead of HDFS.
    
      I've kept the current command line options working, but I changed
      the way configuration is set to be mostly based on SparkConf,
      so that it's easy to support new providers later.
    
    - Make it so the UI for each application is loaded lazily. The
      UIs are cached in memory (cache size configurable) for faster
      subsequent access. This means that we don't need a limit for
      the number of applications listed; the list should fit
      comfortably in memory (since it holds much less data).
    
      Because of this I lowered the number of applications kept in
      memory to 50 (since that setting doesn't influence the number
      of apps listed anymore).
    
    Later, we may want to provide paging in the listing UI, and also
    spilling the listing to disk and loading it on demand to avoid
    large memory usage / slow startup.

commit bda2fa14142cc9dee5b309092c649eac152931b1
Author: Marcelo Vanzin <va...@cloudera.com>
Date:   2014-05-08T21:48:04Z

    Rudimentary paging support for the history UI.
    
    The provider's list api was tweaked a little bit so that the caller
    can get an atomic view of the data currently held in the provider.

commit eee2f5a5c500a74dd0c9fe454b3d91635b61fc25
Author: Marcelo Vanzin <va...@cloudera.com>
Date:   2014-05-09T20:39:26Z

    Ensure server.stop() is called when shutting down.
    
    Also remove the cleanup code from the fs provider. It would be
    better to clean up, but there's a race between that code's cleanup
    and Hadoop's shutdown hook, which closes all file systems kept in
    the cache. So if you try to clean up the fs provider in a shut
    down hook, you may end up with ugly exceptions in the output.
    
    But leave the stop() functionality around in case it's useful for
    future provider implementations.

commit 6fbe0d8d41c6a95a3e1a8db4445359afa134ccd8
Author: Marcelo Vanzin <va...@cloudera.com>
Date:   2014-05-09T22:00:45Z

    Better handle failures when loading app info.
    
    Instead of failing to load all the applications, just
    ignore the one that failed.

commit 91e96ca81ccfe8b57ed4b89bdb2db97ec31f80bb
Author: Marcelo Vanzin <va...@cloudera.com>
Date:   2014-05-09T22:27:04Z

    Fix scalastyle issues.

commit 49d2fd3227d63dfe800981f1a9d4adbec8a05c2e
Author: Marcelo Vanzin <va...@cloudera.com>
Date:   2014-05-09T23:41:15Z

    Fix a comment.

commit e8026f4eeb83152c88ceb87793625bc22d67f362
Author: Marcelo Vanzin <va...@cloudera.com>
Date:   2014-06-02T23:36:42Z

    Review feedback.
    
    Use monotonic time, plus other stylistic things.

commit e8521499051fdef2c091d9f302277b55e3c1f953
Author: Marcelo Vanzin <va...@cloudera.com>
Date:   2014-06-02T23:56:23Z

    Initialize new app array to expected size.
    
    To avoid reallocations.

commit 4406f6159f7dcc620c23031c5f562c8380db1851
Author: Marcelo Vanzin <va...@cloudera.com>
Date:   2014-06-03T17:34:47Z

    Cosmetic change to listing header.

commit b2c570ad0c16bb1296f01e996d04671a17cce421
Author: Marcelo Vanzin <va...@cloudera.com>
Date:   2014-06-03T17:58:31Z

    Make class package-private.

commit 6e2432fc5ad29e05b5d446fa6e940256587c64ee
Author: Marcelo Vanzin <va...@cloudera.com>
Date:   2014-06-04T17:42:07Z

    Second round of feedback.
    
    - Simplify some mt code.
    - Remove new argument that wasn't in 1.0.0, reword some comments.

commit ca5d3200d24ae4cc35f1e6b4e60593ec59144bad
Author: Marcelo Vanzin <va...@cloudera.com>
Date:   2014-06-04T18:02:20Z

    Remove code that deals with unfinished apps.
    
    The HS only reads logs from finished apps, so remove the code
    that checked whether the app was actually finished.

commit 249bcea9f32fc19cb40501a73b30a15adaf4858e
Author: Marcelo Vanzin <va...@cloudera.com>
Date:   2014-06-04T20:31:27Z

    Remove offset / count from provider interface.

commit 4e72c771da6759db0d3b7a9b27170a5f8bf40dc5
Author: Marcelo Vanzin <va...@cloudera.com>
Date:   2014-06-04T20:34:28Z

    Remove comment about ordering.
    
    While ordering is nice to have, it's hard to guarantee that, even with
    ordering, the listing won't change between two client requests (and
    thus end up with different info when the UI applies the paging
    parameters). So don't make it a requirement (even if an informal one).

commit 2a7f68d6d2fa7a9b259f00130e1bf1adce91fb29
Author: Marcelo Vanzin <va...@cloudera.com>
Date:   2014-06-10T00:25:02Z

    Address review feedback.
    
    Main changes:
    - Restore old command line handling.
    - Fix pagination.
    - Restore showing the log directory in the listing page.

commit 4da3a525060e537311439886d2a0bc2b71d2439c
Author: Marcelo Vanzin <va...@cloudera.com>
Date:   2014-06-10T18:52:57Z

    Remove UI from ApplicationHistoryInfo.
    
    This reduces the needed memory when lots of applications are listed,
    since there were 2 pointers wasted per entry to hold UI-specific
    information.

commit dd8cc4b6af0d739fdefbb59857855453bb4177c3
Author: Marcelo Vanzin <va...@cloudera.com>
Date:   2014-06-10T22:13:42Z

    Standardize on using spark.history.* configuration.
    
    Update documentation to mention the config options instead of the old
    command line argument, and update the startup script.

commit c21f8d84bb621dc788df5bad70012f71180a9873
Author: Marcelo Vanzin <va...@cloudera.com>
Date:   2014-06-11T17:29:35Z

    Feedback: formatting, docs.

commit 53620c9ace474ebf5cf733dc9c87ce24c3538edc
Author: Marcelo Vanzin <va...@cloudera.com>
Date:   2014-06-17T17:55:51Z

    Add mima exclude, fix scaladoc wording.

----


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13571516
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -290,8 +212,81 @@ object HistoryServer {
         }
       }
     
    +  private def parse(args: List[String]): Unit = {
    +    args match {
    +      case ("--dir" | "-d") :: value :: tail =>
    +        set("fs.logDirectory",  value)
    +        parse(tail)
    +
    +      case ("-D") :: opt :: value :: tail =>
    --- End diff --
    
    Ok, I'll remove this new command line handling and revert to the old one.
    
    I find having to use SPARK_HISTORY_OPTS and other environment variables to pass argument to processes kinda hacky, but that can be changed separately.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13667818
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,210 @@
    +/*
    + * 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.io.FileNotFoundException
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval",
    +    conf.getInt("spark.history.updateInterval", 10)) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  @volatile private var appList: Seq[ApplicationHistoryInfo] = Nil
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTimeMs()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing() = appList
    +
    +  override def getAppUI(appId: String): SparkUI = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)._2
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  override def getConfig(): Map[String, String] =
    +    Map(("Event Log Location" -> logDir))
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, by not reading
    +   * applications that haven't been updated since last time the logs were checked.
    +   */
    +  private def checkForLogs() = {
    +    lastLogCheckTimeMs = getMonotonicTimeMs()
    +    logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs))
    +    try {
    +      val logStatus = fs.listStatus(new Path(logDir))
    +      val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
    +      val logInfos = logDirs
    +        .filter {
    +            dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))
    +          }
    +
    +      val currentApps = Map[String, ApplicationHistoryInfo](
    +        appList.map(app => (app.id -> app)):_*)
    +
    +      // For any application that either (i) is not listed or (ii) has changed since the last time
    +      // the listing was created (defined by the log dir's modification time), load the app's info.
    +      // Otherwise just reuse what's already in memory.
    +      val newApps = new mutable.ArrayBuffer[ApplicationHistoryInfo](logInfos.size)
    +      for (dir <- logInfos) {
    +        val curr = currentApps.getOrElse(dir.getPath().getName(), null)
    +        if (curr == null || curr.lastUpdated < getModificationTime(dir)) {
    +          try {
    +            newApps += loadAppInfo(dir, false)._1
    +          } catch {
    +            case e: Exception => logError(s"Failed to load app info from directory $dir.")
    +          }
    +        } else {
    +          newApps += curr
    +        }
    +      }
    +
    +      appList = newApps.sortBy { info => -info.endTime }
    +    } catch {
    +      case t: Throwable => logError("Exception in checking for event log updates", t)
    +    }
    +  }
    +
    +  /**
    +   * Parse the application's logs to find out the information we need to build the
    +   * listing page.
    +   *
    +   * When creating the listing of available apps, there is no need to load the whole UI for the
    +   * application. The UI is requested by the HistoryServer (by calling getAppInfo()) when the user
    +   * clicks on a specific application.
    +   *
    +   * @param logDir Directory with application's log files.
    +   * @param renderUI Whether to create the SparkUI for the application.
    +   * @return A 2-tuple `(app info, ui)`. `ui` will be null if `renderUI` is false.
    --- End diff --
    
    Wait sorry, I'm not following. What I meant was to move L188 - 190 to right after we create the SparkUI is L172. What other method are you referring to?


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13365957
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,203 @@
    +/*
    + * 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.io.FileNotFoundException
    +import java.util.concurrent.atomic.AtomicReference
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval", 10) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  private val appList = new AtomicReference[Seq[ApplicationHistoryInfo]](Nil)
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTime()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing(offset: Int, count: Int) = {
    +    val list = appList.get()
    +    val theOffset = if (offset < list.size) offset else 0
    +    (list.slice(theOffset, Math.min(theOffset + count, list.size)), theOffset, list.size)
    +  }
    +
    +  override def getAppInfo(appId: String): ApplicationHistoryInfo = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, but not reading
    +   * applications that hasn't been updated since last time the logs were checked.
    +   */
    +  def checkForLogs() = synchronized {
    +    lastLogCheckTimeMs = getMonotonicTime()
    +    logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs))
    +    try {
    +      val logStatus = fs.listStatus(new Path(logDir))
    +      val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
    +      val logInfos = logDirs
    +        .filter {
    +            dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))
    +          }
    +
    +      var currentApps = Map[String, ApplicationHistoryInfo](
    +        appList.get().map(app => (app.id -> app)):_*)
    +
    +      // For any application that either (i) is not listed or (ii) has changed since the last time
    +      // the listing was created (defined by the log dir's modification time), load the app's info.
    +      // Otherwise just reuse what's already in memory.
    +      val newApps = new mutable.ArrayBuffer[ApplicationHistoryInfo](logInfos.size)
    +      for (dir <- logInfos) {
    +        val curr = currentApps.getOrElse(dir.getPath().getName(), null)
    +        if (curr == null || curr.lastUpdated < getModificationTime(dir)) {
    +          try {
    +            newApps += loadAppInfo(dir, false)
    +          } catch {
    +            case e: Exception => logError(s"Failed to load app info from directory $dir.")
    +          }
    +        } else {
    +          newApps += curr
    +        }
    +      }
    +
    +      appList.set(newApps.sortBy { info => -info.lastUpdated })
    +    } catch {
    +      case t: Throwable => logError("Exception in checking for event log updates", t)
    +    }
    +  }
    +
    +  /**
    +   * Parse the application's logs to find out the information we need to build the
    +   * listing page.
    +   */
    +  private def loadAppInfo(logDir: FileStatus, renderUI: Boolean): ApplicationHistoryInfo = {
    +    val elogInfo = EventLoggingListener.parseLoggingInfo(logDir.getPath(), fs)
    +    val path = logDir.getPath
    +    val appId = path.getName
    +    val replayBus = new ReplayListenerBus(elogInfo.logPaths, fs, elogInfo.compressionCodec)
    +    val appListener = new ApplicationEventListener
    +    replayBus.addListener(appListener)
    +
    +    val ui: SparkUI = if (renderUI) {
    +        val conf = this.conf.clone()
    +        val appSecManager = new SecurityManager(conf)
    +        new SparkUI(conf, appSecManager, replayBus, appId, "/history/" + appId)
    +        // Do not call ui.bind() to avoid creating a new server for each application
    +      } else {
    +        null
    +      }
    +
    +    replayBus.replay()
    +    val appName = appListener.appName
    +    val sparkUser = appListener.sparkUser
    +    val startTime = appListener.startTime
    +    val endTime = appListener.endTime
    +    val lastUpdated = getModificationTime(logDir)
    +    ApplicationHistoryInfo(appId,
    +      appListener.appName,
    +      appListener.startTime,
    +      appListener.endTime,
    +      getModificationTime(logDir),
    +      appListener.sparkUser,
    +      if (renderUI) appListener.viewAcls else null,
    +      ui)
    +  }
    +
    +  /** Return when this directory was last modified. */
    +  private def getModificationTime(dir: FileStatus): Long = {
    +    try {
    +      val logFiles = fs.listStatus(dir.getPath)
    +      if (logFiles != null && !logFiles.isEmpty) {
    +        logFiles.map(_.getModificationTime).max
    +      } else {
    +        dir.getModificationTime
    +      }
    +    } catch {
    +      case t: Throwable =>
    +        logError("Exception in accessing modification time of %s".format(dir.getPath), t)
    +        -1L
    +    }
    +  }
    +
    +  /** Returns the system's mononotically increasing time. */
    +  private def getMonotonicTime() = System.nanoTime() / (1000 * 1000)
    --- End diff --
    
    Isn't time always monotonic? Also, is there any reason not to just do `System.currentTimeMillis()` if you want milliseconds? Do these have different behavior?


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13564401
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -290,8 +212,81 @@ object HistoryServer {
         }
       }
     
    +  private def parse(args: List[String]): Unit = {
    +    args match {
    +      case ("--dir" | "-d") :: value :: tail =>
    +        set("fs.logDirectory",  value)
    +        parse(tail)
    +
    +      case ("-D") :: opt :: value :: tail =>
    +        set(opt, value)
    +        parse(tail)
    +
    +      case ("--help" | "-h") :: tail =>
    +        printUsageAndExit(0)
    +
    +      case Nil =>
    +
    +      case _ =>
    +        printUsageAndExit(1)
    +    }
    +  }
    +
    +  private def set(name: String, value: String) = {
    +    conf.set("spark.history." + name, value)
    +  }
    +
    +  private def printUsageAndExit(exitCode: Int) {
    +    System.err.println(
    +      """
    +      |Usage: HistoryServer [options]
    +      |
    +      |Options are set by passing "-D option value" command line arguments to the class.
    +      |Command line options will override JVM system properties (which should be prepended
    +      |with "spark.history.").
    +      |
    +      |History Server options are always available; additional options depend on the provider.
    +      |
    +      |History Server options:
    +      |
    +      |  ui.port           Port where server will listen for connections (default 18080)
    +      |  ui.acls.enable    Whether to enable view acls for all applications (default false)
    +      |  provider          Name of history provider class (defaults to file system-based provider)
    +      |
    +      |FsHistoryProvider options:
    +      |
    +      |  fs.logDirectory   Directory where app logs are stored (required)
    +      |  fs.updateInterval How often to reload log data from storage (seconds, default 10)
    --- End diff --
    
    as mentioned above, these should all be prefixed with `spark.history.*`


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-45023292
  
    BTW, I have some changes on top of this one I've been working on, but don't want to pile on top of this same PR. If you'd like to take a look and comment, here's the first:
    
    https://github.com/vanzin/spark/pull/1
    
    (I don't know how to make github send an incremental pr so that only the new commits from the pr show up here, if anyone has any hints that would be appreciated.)


---
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-1768] History server enhancements.

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

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


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13397985
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,203 @@
    +/*
    + * 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.io.FileNotFoundException
    +import java.util.concurrent.atomic.AtomicReference
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval", 10) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  private val appList = new AtomicReference[Seq[ApplicationHistoryInfo]](Nil)
    --- End diff --
    
    It's not a performance thing, but a correctness thing, since the variable is accessed by different threads (one of which might be writing to it). You really don't two threads, one adding things to an ArrayBuffer while the other is trying to iterate through it, for example. That will end up in tears.
    
    That being said, since the code is only reading and writing to the variable (and not making other kinds of non-atomic changes), volatile is probably enough. A lock is way overkill 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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13365803
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,203 @@
    +/*
    + * 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.io.FileNotFoundException
    +import java.util.concurrent.atomic.AtomicReference
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval", 10) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  private val appList = new AtomicReference[Seq[ApplicationHistoryInfo]](Nil)
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTime()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing(offset: Int, count: Int) = {
    +    val list = appList.get()
    +    val theOffset = if (offset < list.size) offset else 0
    +    (list.slice(theOffset, Math.min(theOffset + count, list.size)), theOffset, list.size)
    +  }
    +
    +  override def getAppInfo(appId: String): ApplicationHistoryInfo = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, but not reading
    +   * applications that hasn't been updated since last time the logs were checked.
    +   */
    +  def checkForLogs() = synchronized {
    +    lastLogCheckTimeMs = getMonotonicTime()
    +    logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs))
    +    try {
    +      val logStatus = fs.listStatus(new Path(logDir))
    +      val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
    +      val logInfos = logDirs
    +        .filter {
    +            dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))
    +          }
    +
    +      var currentApps = Map[String, ApplicationHistoryInfo](
    --- End diff --
    
    This could be a val - you never modify 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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-46382733
  
    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-1768] History server enhancements.

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

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


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13564288
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -290,8 +212,81 @@ object HistoryServer {
         }
       }
     
    +  private def parse(args: List[String]): Unit = {
    +    args match {
    +      case ("--dir" | "-d") :: value :: tail =>
    +        set("fs.logDirectory",  value)
    +        parse(tail)
    +
    +      case ("-D") :: opt :: value :: tail =>
    --- End diff --
    
    What options do you envision the user pass in? Are you thinking about `spark.history.*`? If so, these should be set through SPARK_HISTORY_OPTS in spark-env.sh, not through the `-D` option on the command line.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-46373773
  
     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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13630958
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -38,56 +39,70 @@ import org.apache.spark.util.Utils
      * 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,
    +    provider: ApplicationHistoryProvider,
         securityManager: SecurityManager,
    -    conf: SparkConf)
    -  extends WebUI(securityManager, HistoryServer.WEB_UI_PORT, conf) with Logging {
    +    port: Int)
    +  extends WebUI(securityManager, port, conf) with Logging {
     
    -  import HistoryServer._
    +  // How many applications to retain
    +  private val retainedApplications = conf.getInt("spark.history.retainedApplications", 50)
     
    -  private val fileSystem = Utils.getHadoopFileSystem(baseLogDir)
       private val localHost = Utils.localHostName()
    -  private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHost)
     
    -  // A timestamp of when the disk was last accessed to check for log updates
    -  private var lastLogCheckTime = -1L
    +  private val appLoader = new CacheLoader[String, SparkUI] {
    +    override def load(key: String): SparkUI = {
    +      val ui = provider.getAppUI(key)
    +      if (ui == null) {
    +        throw new NoSuchElementException()
    +      }
    +      attachSparkUI(ui)
    +      ui
    +    }
    +  }
     
    -  // Number of completed applications found in this directory
    -  private var numCompletedApplications = 0
    +  private val appCache = CacheBuilder.newBuilder()
    +    .maximumSize(retainedApplications)
    +    .removalListener(new RemovalListener[String, SparkUI] {
    +      override def onRemoval(rm: RemovalNotification[String, SparkUI]) = {
    +        detachSparkUI(rm.getValue())
    +      }
    +    })
    +    .build(appLoader)
    +
    +  private val loaderServlet = new HttpServlet {
    +    protected override def doGet(req: HttpServletRequest, res: HttpServletResponse): Unit = {
    +      val parts = Option(req.getPathInfo()).getOrElse("").split("/")
    +      if (parts.length < 2) {
    +        res.sendError(HttpServletResponse.SC_BAD_REQUEST,
    +          s"Unexpected path info in request (URI = ${req.getRequestURI()}")
    --- End diff --
    
    Oh I see, they will see it on the actual page. Then it's fine.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13566297
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,193 @@
    +/*
    + * 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.io.FileNotFoundException
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval", 10) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  @volatile private var appList: Seq[ApplicationHistoryInfo] = Nil
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTime()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing() = appList
    +
    +  override def getAppInfo(appId: String): ApplicationHistoryInfo = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, by not reading
    +   * applications that haven't been updated since last time the logs were checked.
    +   */
    +  def checkForLogs() = {
    +    lastLogCheckTimeMs = getMonotonicTime()
    +    logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs))
    +    try {
    +      val logStatus = fs.listStatus(new Path(logDir))
    +      val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
    +      val logInfos = logDirs
    +        .filter {
    +            dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))
    +          }
    +
    +      val currentApps = Map[String, ApplicationHistoryInfo](
    +        appList.map(app => (app.id -> app)):_*)
    +
    +      // For any application that either (i) is not listed or (ii) has changed since the last time
    +      // the listing was created (defined by the log dir's modification time), load the app's info.
    +      // Otherwise just reuse what's already in memory.
    +      val newApps = new mutable.ArrayBuffer[ApplicationHistoryInfo](logInfos.size)
    +      for (dir <- logInfos) {
    +        val curr = currentApps.getOrElse(dir.getPath().getName(), null)
    +        if (curr == null || curr.lastUpdated < getModificationTime(dir)) {
    +          try {
    +            newApps += loadAppInfo(dir, false)
    +          } catch {
    +            case e: Exception => logError(s"Failed to load app info from directory $dir.")
    +          }
    +        } else {
    +          newApps += curr
    +        }
    +      }
    +
    +      appList = newApps.sortBy { info => -info.endTime }
    +    } catch {
    +      case t: Throwable => logError("Exception in checking for event log updates", t)
    +    }
    +  }
    +
    +  /**
    +   * Parse the application's logs to find out the information we need to build the
    +   * listing page.
    --- End diff --
    
    Would be good to document here when we want the parameter `renderUI` to be true, and when we want it to be false.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13365793
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,203 @@
    +/*
    + * 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.io.FileNotFoundException
    +import java.util.concurrent.atomic.AtomicReference
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval", 10) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  private val appList = new AtomicReference[Seq[ApplicationHistoryInfo]](Nil)
    --- End diff --
    
    Is there a reason not to just use an `ArrayBuffer` here and add some synchronization around resetting and reading it? This `AtomicReference` thing is a little bit clunky and I don't see any performance argument for 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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13630600
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -251,30 +174,31 @@ class HistoryServer(
     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
    -
    -  // How many applications to retain
    -  val RETAINED_APPLICATIONS = conf.getInt("spark.history.retainedApplications", 250)
    -
    -  // The port to which the web UI is bound
    -  val WEB_UI_PORT = conf.getInt("spark.history.ui.port", 18080)
    -
    -  // set whether to enable or disable view acls for all applications
    -  val HISTORY_UI_ACLS_ENABLED = conf.getBoolean("spark.history.ui.acls.enable", false)
    -
    -  val STATIC_RESOURCE_DIR = SparkUI.STATIC_RESOURCE_DIR
    -
       def main(argStrings: Array[String]) {
         initSecurity()
    -    val args = new HistoryServerArguments(argStrings)
    +    val args = new HistoryServerArguments(conf, argStrings)
         val securityManager = new SecurityManager(conf)
    -    val server = new HistoryServer(args.logDir, securityManager, conf)
    +
    +    val providerName = conf.getOption("spark.history.provider")
    +      .getOrElse(classOf[FsHistoryProvider].getName())
    +    val provider = Class.forName(providerName)
    +      .getConstructor(classOf[SparkConf])
    +      .newInstance(conf)
    +      .asInstanceOf[ApplicationHistoryProvider]
    +
    +    val port = conf.getInt("spark.history.ui.port", 18080)
    +
    +    val server = new HistoryServer(conf, provider, securityManager, port)
         server.bind()
     
    +    Runtime.getRuntime().addShutdownHook(new Thread("HistoryServerStopper") {
    +        override def run() = {
    +          server.stop()
    +        }
    +      })
    --- End diff --
    
    minor: unindent these lines


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13566154
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,193 @@
    +/*
    + * 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.io.FileNotFoundException
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval", 10) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  @volatile private var appList: Seq[ApplicationHistoryInfo] = Nil
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTime()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing() = appList
    +
    +  override def getAppInfo(appId: String): ApplicationHistoryInfo = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, by not reading
    +   * applications that haven't been updated since last time the logs were checked.
    +   */
    +  def checkForLogs() = {
    +    lastLogCheckTimeMs = getMonotonicTime()
    +    logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs))
    +    try {
    +      val logStatus = fs.listStatus(new Path(logDir))
    +      val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
    +      val logInfos = logDirs
    +        .filter {
    +            dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))
    +          }
    +
    +      val currentApps = Map[String, ApplicationHistoryInfo](
    +        appList.map(app => (app.id -> app)):_*)
    --- End diff --
    
    Looks like we create a map every time we check for logs. This is quite expensive if there are many applications. How about just storing them as a map 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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-46767582
  
    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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-45550223
  
    @vanzin Thanks for your work on the history server. I left a few comments in the code.
    
    It seems that right now we replay every rendered application twice (by calling `loadAppInfo`), once when we first check for the logs, and once when we actually render it. This could be expensive if these logs are huge. Is there a way to avoid this? It looks like we're only really interested in the ApplicationStart event anyway if we're not actually rendering the SparkUI. This probably requires a little bit of refactoring.
    
    Also, I tested the pagination thing locally and it didn't work for me. When I tried to load anything beyond the first page, I got blank tables. Are you able to reproduce 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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13566891
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala ---
    @@ -25,20 +25,32 @@ import org.apache.spark.ui.{WebUIPage, UIUtils}
     
     private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") {
     
    +  val pageSize = 20
    +
       def render(request: HttpServletRequest): Seq[Node] = {
    -    val appRows = parent.appIdToInfo.values.toSeq.sortBy { app => -app.lastUpdated }
    -    val appTable = UIUtils.listingTable(appHeader, appRow, appRows)
    +    val requestedPage = Option(request.getParameter("page")).getOrElse("1").toInt
    +    val requestedFirst = (requestedPage - 1) * pageSize
    +
    +    val allApps = parent.getApplicationList()
    +    val actualFirst = if (requestedFirst < allApps.size) requestedFirst else 0
    +    val apps = allApps.slice(actualFirst, Math.min(pageSize, allApps.size))
    +
    +    val actualPage = (actualFirst / pageSize) + 1
    +    val last = Math.min(actualFirst + pageSize, allApps.size) - 1
    +    val pageCount = allApps.size / pageSize + (if (allApps.size % pageSize > 0) 1 else 0)
    +
    +    val appTable = UIUtils.listingTable(appHeader, appRow, apps)
         val content =
           <div class="row-fluid">
             <div class="span12">
    -          <ul class="unstyled">
    -            <li><strong>Event Log Location: </strong> {parent.baseLogDir}</li>
    --- End diff --
    
    Why was this removed? This is useful to check whether we specified the correct logging directory when we started the history server.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13366328
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -290,8 +217,88 @@ object HistoryServer {
         }
       }
     
    +  private def parse(args: List[String]): Unit = {
    +    args match {
    +      case ("--dir" | "-d") :: value :: tail =>
    +        set("fs.logDirectory",  value)
    +        parse(tail)
    +
    +      case ("--port" | "-p") :: value :: tail =>
    +        set("ui.port", value)
    +        parse(tail)
    +
    +      case ("-D") :: opt :: value :: tail =>
    +        set(opt, value)
    +        parse(tail)
    +
    +      case ("--help" | "-h") :: tail =>
    +        printUsageAndExit(0)
    +
    +      case Nil =>
    +
    +      case _ =>
    +        printUsageAndExit(1)
    +    }
    +  }
    +
    +  private def set(name: String, value: String) = {
    +    conf.set("spark.history." + name, value)
    +  }
    +
    +  private def printUsageAndExit(exitCode: Int) {
    +    System.err.println(
    +      """
    +      |Usage: HistoryServer [options]
    +      |
    +      |Options are set by passing "-D option value" command line arguments to the class.
    +      |Command line options will override the Spark configuration file and system properties.
    --- End diff --
    
    I don't think the Spark configuration file would ever be read here (presently that file is just for `spark-submit`). I would just say it overrides system properties.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13666912
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,210 @@
    +/*
    + * 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.io.FileNotFoundException
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval",
    +    conf.getInt("spark.history.updateInterval", 10)) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  @volatile private var appList: Seq[ApplicationHistoryInfo] = Nil
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTimeMs()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing() = appList
    +
    +  override def getAppUI(appId: String): SparkUI = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)._2
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  override def getConfig(): Map[String, String] =
    +    Map(("Event Log Location" -> logDir))
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, by not reading
    +   * applications that haven't been updated since last time the logs were checked.
    +   */
    +  private def checkForLogs() = {
    +    lastLogCheckTimeMs = getMonotonicTimeMs()
    +    logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs))
    +    try {
    +      val logStatus = fs.listStatus(new Path(logDir))
    +      val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
    +      val logInfos = logDirs
    +        .filter {
    +            dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))
    +          }
    +
    +      val currentApps = Map[String, ApplicationHistoryInfo](
    +        appList.map(app => (app.id -> app)):_*)
    +
    +      // For any application that either (i) is not listed or (ii) has changed since the last time
    +      // the listing was created (defined by the log dir's modification time), load the app's info.
    +      // Otherwise just reuse what's already in memory.
    +      val newApps = new mutable.ArrayBuffer[ApplicationHistoryInfo](logInfos.size)
    +      for (dir <- logInfos) {
    +        val curr = currentApps.getOrElse(dir.getPath().getName(), null)
    +        if (curr == null || curr.lastUpdated < getModificationTime(dir)) {
    +          try {
    +            newApps += loadAppInfo(dir, false)._1
    +          } catch {
    +            case e: Exception => logError(s"Failed to load app info from directory $dir.")
    +          }
    +        } else {
    +          newApps += curr
    +        }
    +      }
    +
    +      appList = newApps.sortBy { info => -info.endTime }
    +    } catch {
    +      case t: Throwable => logError("Exception in checking for event log updates", t)
    +    }
    +  }
    +
    +  /**
    +   * Parse the application's logs to find out the information we need to build the
    +   * listing page.
    +   *
    +   * When creating the listing of available apps, there is no need to load the whole UI for the
    +   * application. The UI is requested by the HistoryServer (by calling getAppInfo()) when the user
    +   * clicks on a specific application.
    +   *
    +   * @param logDir Directory with application's log files.
    +   * @param renderUI Whether to create the SparkUI for the application.
    +   * @return A 2-tuple `(app info, ui)`. `ui` will be null if `renderUI` is false.
    --- End diff --
    
    The issue is not having the SparkListenerBus as part of the constructor. Even if SparkUI has a `registerListeners` interface, we still need to replay the events after the SparkUI has registered all of its listeners to the ReplayBus. In either case, we need to replay only after SparkUI has been instantiated and its listeners attached to the bus.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13367318
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -290,8 +217,88 @@ object HistoryServer {
         }
       }
     
    +  private def parse(args: List[String]): Unit = {
    +    args match {
    +      case ("--dir" | "-d") :: value :: tail =>
    +        set("fs.logDirectory",  value)
    +        parse(tail)
    +
    +      case ("--port" | "-p") :: value :: tail =>
    +        set("ui.port", value)
    +        parse(tail)
    +
    +      case ("-D") :: opt :: value :: tail =>
    +        set(opt, value)
    +        parse(tail)
    +
    +      case ("--help" | "-h") :: tail =>
    +        printUsageAndExit(0)
    +
    +      case Nil =>
    +
    +      case _ =>
    +        printUsageAndExit(1)
    +    }
    +  }
    +
    +  private def set(name: String, value: String) = {
    +    conf.set("spark.history." + name, value)
    +  }
    +
    +  private def printUsageAndExit(exitCode: Int) {
    +    System.err.println(
    +      """
    +      |Usage: HistoryServer [options]
    +      |
    +      |Options are set by passing "-D option value" command line arguments to the class.
    +      |Command line options will override the Spark configuration file and system properties.
    +      |History Server options are always available; additional options depend on the provider.
    +      |
    +      |History Server options:
    +      |
    +      |  ui.port           Port where server will listen for connections (default 18080)
    +      |  ui.acls.enable    Whether to enable view acls for all applications (default false)
    +      |  provider          Name of history provider class (defaults to file system-based provider)
    +      |
    +      |FsHistoryProvider options:
    +      |
    +      |  fs.logDirectory   Directory where app logs are stored (required)
    +      |  fs.updateInterval How often to reload log data from storage (seconds, default 10)
    +      |""".stripMargin)
    +    System.exit(exitCode)
    +  }
    +
     }
     
    +private[spark] abstract class ApplicationHistoryProvider {
    +
    +  /**
    +   * This method should return a list of applications available for the history server to
    +   * show. The listing is assumed to be in descending time order.
    +   *
    +   * An adjusted offset should be returned if the app list has changed and the request
    +   * references an invalid start offset. Otherwise, the provided offset should be returned.
    +   *
    +   * @param offset Starting offset for returned objects.
    +   * @param count Max number of objects to return.
    +   * @return 3-tuple (requested app list, adjusted offset, count of all available apps)
    +   */
    +  def getListing(offset: Int, count: Int): (Seq[ApplicationHistoryInfo], Int, Int)
    --- End diff --
    
    As far as I can tell, there is no e.g. performance benefit of only asking for a range (under the hood it's just reading from the entire list 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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13619055
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,210 @@
    +/*
    + * 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.io.FileNotFoundException
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval",
    +    conf.getInt("spark.history.updateInterval", 10)) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  @volatile private var appList: Seq[ApplicationHistoryInfo] = Nil
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTimeMs()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing() = appList
    +
    +  override def getAppUI(appId: String): SparkUI = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)._2
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  override def getConfig(): Map[String, String] =
    +    Map(("Event Log Location" -> logDir))
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, by not reading
    +   * applications that haven't been updated since last time the logs were checked.
    +   */
    +  private def checkForLogs() = {
    +    lastLogCheckTimeMs = getMonotonicTimeMs()
    +    logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs))
    +    try {
    +      val logStatus = fs.listStatus(new Path(logDir))
    +      val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
    +      val logInfos = logDirs
    +        .filter {
    +            dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))
    +          }
    +
    +      val currentApps = Map[String, ApplicationHistoryInfo](
    +        appList.map(app => (app.id -> app)):_*)
    +
    +      // For any application that either (i) is not listed or (ii) has changed since the last time
    +      // the listing was created (defined by the log dir's modification time), load the app's info.
    +      // Otherwise just reuse what's already in memory.
    +      val newApps = new mutable.ArrayBuffer[ApplicationHistoryInfo](logInfos.size)
    +      for (dir <- logInfos) {
    +        val curr = currentApps.getOrElse(dir.getPath().getName(), null)
    +        if (curr == null || curr.lastUpdated < getModificationTime(dir)) {
    +          try {
    +            newApps += loadAppInfo(dir, false)._1
    +          } catch {
    +            case e: Exception => logError(s"Failed to load app info from directory $dir.")
    +          }
    +        } else {
    +          newApps += curr
    +        }
    +      }
    +
    +      appList = newApps.sortBy { info => -info.endTime }
    +    } catch {
    +      case t: Throwable => logError("Exception in checking for event log updates", t)
    +    }
    +  }
    +
    +  /**
    +   * Parse the application's logs to find out the information we need to build the
    +   * listing page.
    +   *
    +   * When creating the listing of available apps, there is no need to load the whole UI for the
    +   * application. The UI is requested by the HistoryServer (by calling getAppInfo()) when the user
    +   * clicks on a specific application.
    +   *
    +   * @param logDir Directory with application's log files.
    +   * @param renderUI Whether to create the SparkUI for the application.
    +   * @return A 2-tuple `(app info, ui)`. `ui` will be null if `renderUI` is false.
    --- End diff --
    
    For instance,
    
    ```
    override def getAppUI(logDir): SparkUI = {
      val eventLogInfo = // ... parse log info
      val replayBus = new ReplayListenerBus(...)
      val ui = new SparkUI(...)
      replayBus.replay()
      // ... set ACLs appropriately
      ui
    }
    
    private def loadAppInfo(logDir: FileStatus): ApplicationHistoryInfo = {
      val eventLogInfo = // ... parse log info
      val replayBus = new ReplayListenerBus(...)
      replayBus.replay()
      // ... other application history info
      new ApplicationHistoryInfo(...)
    }
    ```
    
    That way you don't have to check if `ui` is null.



---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13367189
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -290,8 +217,88 @@ object HistoryServer {
         }
       }
     
    +  private def parse(args: List[String]): Unit = {
    +    args match {
    +      case ("--dir" | "-d") :: value :: tail =>
    +        set("fs.logDirectory",  value)
    +        parse(tail)
    +
    +      case ("--port" | "-p") :: value :: tail =>
    +        set("ui.port", value)
    +        parse(tail)
    +
    +      case ("-D") :: opt :: value :: tail =>
    --- End diff --
    
    I think this was somewhat intentionally designed to only take a single required argument. Checkout the existing docs for the history server:
    http://spark.apache.org/docs/latest/monitoring.html
    
    Doesn't this also break compatibility with 1.0 as well?
    
    Separately, I'd rather not overload `-D` in general. This seems to actually have different syntax than the normal `-D` where you do `-Dkey=value`. Here it's `-D key value`.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-46340917
  
    LGTM pending getting the MIMA stuff working.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-46373331
  
    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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13661671
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,210 @@
    +/*
    + * 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.io.FileNotFoundException
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval",
    +    conf.getInt("spark.history.updateInterval", 10)) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  @volatile private var appList: Seq[ApplicationHistoryInfo] = Nil
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTimeMs()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing() = appList
    +
    +  override def getAppUI(appId: String): SparkUI = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)._2
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  override def getConfig(): Map[String, String] =
    +    Map(("Event Log Location" -> logDir))
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, by not reading
    +   * applications that haven't been updated since last time the logs were checked.
    +   */
    +  private def checkForLogs() = {
    +    lastLogCheckTimeMs = getMonotonicTimeMs()
    +    logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs))
    +    try {
    +      val logStatus = fs.listStatus(new Path(logDir))
    +      val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
    +      val logInfos = logDirs
    +        .filter {
    +            dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))
    +          }
    +
    +      val currentApps = Map[String, ApplicationHistoryInfo](
    +        appList.map(app => (app.id -> app)):_*)
    +
    +      // For any application that either (i) is not listed or (ii) has changed since the last time
    +      // the listing was created (defined by the log dir's modification time), load the app's info.
    +      // Otherwise just reuse what's already in memory.
    +      val newApps = new mutable.ArrayBuffer[ApplicationHistoryInfo](logInfos.size)
    +      for (dir <- logInfos) {
    +        val curr = currentApps.getOrElse(dir.getPath().getName(), null)
    +        if (curr == null || curr.lastUpdated < getModificationTime(dir)) {
    +          try {
    +            newApps += loadAppInfo(dir, false)._1
    +          } catch {
    +            case e: Exception => logError(s"Failed to load app info from directory $dir.")
    +          }
    +        } else {
    +          newApps += curr
    +        }
    +      }
    +
    +      appList = newApps.sortBy { info => -info.endTime }
    +    } catch {
    +      case t: Throwable => logError("Exception in checking for event log updates", t)
    +    }
    +  }
    +
    +  /**
    +   * Parse the application's logs to find out the information we need to build the
    +   * listing page.
    +   *
    +   * When creating the listing of available apps, there is no need to load the whole UI for the
    +   * application. The UI is requested by the HistoryServer (by calling getAppInfo()) when the user
    +   * clicks on a specific application.
    +   *
    +   * @param logDir Directory with application's log files.
    +   * @param renderUI Whether to create the SparkUI for the application.
    +   * @return A 2-tuple `(app info, ui)`. `ui` will be null if `renderUI` is false.
    --- End diff --
    
    The problem is that what you suggest is just not possible, because you need to replay bus to be instantiated before the UI, and replayed after the UI is instantiated.
    
    So you have 3 options:
    - duplicate the code, which I don't like
    - my approach, which is ugly but my least hated of all the options
    - break things down into even more methods:
      * one to instantiate the bus
      * one to replay the bus
      * one for building the UI, one for building just the info, both of which call the two above.
    
    The last approach, which is the closes to what you suggest above, would have more noise (method declarations et al) than actual code.
    
    The *real* fix would be to fix the SparkUI signature so that it makes sense (e,.g. make it act like a proper listener instead of doing what it currently does), but that's way beyond the scope of 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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13565393
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -38,56 +39,75 @@ import org.apache.spark.util.Utils
      * 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,
    +    provider: ApplicationHistoryProvider,
         securityManager: SecurityManager,
    -    conf: SparkConf)
    -  extends WebUI(securityManager, HistoryServer.WEB_UI_PORT, conf) with Logging {
    +    port: Int)
    +  extends WebUI(securityManager, port, conf) with Logging {
    +
    +  // How many applications to retain
    +  private val retainedApplications = conf.getInt("spark.history.retainedApplications", 50)
     
    -  import HistoryServer._
    +  // set whether to enable or disable view acls for all applications
    +  private val uiAclsEnabled = conf.getBoolean("spark.history.ui.acls.enable", false)
     
    -  private val fileSystem = Utils.getHadoopFileSystem(baseLogDir)
       private val localHost = Utils.localHostName()
       private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHost)
     
    -  // A timestamp of when the disk was last accessed to check for log updates
    -  private var lastLogCheckTime = -1L
    +  private val appLoader = new CacheLoader[String, SparkUI] {
    +    override def load(key: String): SparkUI = {
    +      val info = provider.getAppInfo(key)
    +      if (info == null) {
    +        throw new NoSuchElementException()
    +      }
    +      info.ui.getSecurityManager.setUIAcls(uiAclsEnabled)
    +      info.ui.getSecurityManager.setViewAcls(info.sparkUser, info.viewAcls)
    +      attachSparkUI(info.ui)
    +      info.ui
    +    }
    +  }
     
    -  // Number of completed applications found in this directory
    -  private var numCompletedApplications = 0
    +  private val appCache = CacheBuilder.newBuilder()
    +    .maximumSize(retainedApplications)
    +    .removalListener(new RemovalListener[String, SparkUI] {
    +      override def onRemoval(rm: RemovalNotification[String, SparkUI]) = {
    +        detachSparkUI(rm.getValue())
    +      }
    +    })
    +    .build(appLoader)
    +
    +  private val loaderServlet = new HttpServlet {
    +    protected override def doGet(req: HttpServletRequest, res: HttpServletResponse): Unit = {
    +      val parts = req.getPathInfo().split("/")
    +      if (parts.length < 2) {
    +        res.setStatus(HttpServletResponse.SC_BAD_REQUEST)
    +        return
    +      }
     
    -  @volatile private var stopped = false
    +      var appId = parts(1)
    --- End diff --
    
    This should be a val


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13564133
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -290,8 +212,81 @@ object HistoryServer {
         }
       }
     
    +  private def parse(args: List[String]): Unit = {
    --- End diff --
    
    I would prefer this to be moved back to HistoryServerArguments.scala, which you removed. This is consistent with the standalone master web UI (MasterArguments.scala) and spark submit (SparkSubmitArguments.scala) among other places, and just keeps the main HistoryServer.scala smaller.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13618519
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala ---
    @@ -67,10 +69,29 @@ private[spark] class HistoryServerArguments(args: Array[String]) {
     
       private def printUsageAndExit(exitCode: Int) {
         System.err.println(
    -      "Usage: HistoryServer [options]\n" +
    -      "\n" +
    -      "Options:\n" +
    -      "  -d DIR,  --dir DIR     Location of event log files")
    +      """
    +      |Usage: HistoryServer [-d logDir]
    +      |
    +      |The preferred way to pass options is to set the configuration below using
    +      |SPARK_HISTORY_OPTS. The "-d" command line argument is avalable for backwards
    +      |compatibility, and overrides "spark.history.fs.logDirectory".
    +      |
    +      |History Server options are always available; additional options depend on the provider.
    +      |
    +      |History Server options:
    +      |
    +      |  spark.history.ui.port      Port where server will listen for connections (default 18080)
    +      |  spark.history.acls.enable  Whether to enable view acls for all applications (default false)
    +      |  spark.history.provider     Name of history provider class (defaults to file system-based
    +      |                             provider)
    +      |
    +      |FsHistoryProvider options:
    +      |
    +      |  spark.history.fs.logDirectory    Directory where app logs are stored (required)
    --- End diff --
    
    This is technically not required, as in the user can pass this through the --dir option rather than through `spark.history.fs.logDirectory`.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13403213
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -290,8 +217,88 @@ object HistoryServer {
         }
       }
     
    +  private def parse(args: List[String]): Unit = {
    +    args match {
    +      case ("--dir" | "-d") :: value :: tail =>
    +        set("fs.logDirectory",  value)
    +        parse(tail)
    +
    +      case ("--port" | "-p") :: value :: tail =>
    +        set("ui.port", value)
    +        parse(tail)
    +
    +      case ("-D") :: opt :: value :: tail =>
    +        set(opt, value)
    +        parse(tail)
    +
    +      case ("--help" | "-h") :: tail =>
    +        printUsageAndExit(0)
    +
    +      case Nil =>
    +
    +      case _ =>
    +        printUsageAndExit(1)
    +    }
    +  }
    +
    +  private def set(name: String, value: String) = {
    +    conf.set("spark.history." + name, value)
    +  }
    +
    +  private def printUsageAndExit(exitCode: Int) {
    +    System.err.println(
    +      """
    +      |Usage: HistoryServer [options]
    +      |
    +      |Options are set by passing "-D option value" command line arguments to the class.
    +      |Command line options will override the Spark configuration file and system properties.
    +      |History Server options are always available; additional options depend on the provider.
    +      |
    +      |History Server options:
    +      |
    +      |  ui.port           Port where server will listen for connections (default 18080)
    +      |  ui.acls.enable    Whether to enable view acls for all applications (default false)
    +      |  provider          Name of history provider class (defaults to file system-based provider)
    +      |
    +      |FsHistoryProvider options:
    +      |
    +      |  fs.logDirectory   Directory where app logs are stored (required)
    +      |  fs.updateInterval How often to reload log data from storage (seconds, default 10)
    +      |""".stripMargin)
    +    System.exit(exitCode)
    +  }
    +
     }
     
    +private[spark] abstract class ApplicationHistoryProvider {
    +
    +  /**
    +   * This method should return a list of applications available for the history server to
    +   * show. The listing is assumed to be in descending time order.
    +   *
    +   * An adjusted offset should be returned if the app list has changed and the request
    +   * references an invalid start offset. Otherwise, the provided offset should be returned.
    +   *
    +   * @param offset Starting offset for returned objects.
    +   * @param count Max number of objects to return.
    +   * @return 3-tuple (requested app list, adjusted offset, count of all available apps)
    +   */
    +  def getListing(offset: Int, count: Int): (Seq[ApplicationHistoryInfo], Int, Int)
    --- End diff --
    
    I meant you have an index function that doesn't return the entire in-memory object, it just returns application ID's and other meta-data.
    
    What is "descending time order"? Is it the time the application was created? The time the application log was created? The time the application log was last updated? Anyways I just don't see any benefit here at all of adding this offset stuff, it just looks ugly and someone is going to come along and wonder why this is here and it won't be easy for them to figure it out.
    
    If we want to optimize this, it's likely we'd add support for some sort of predicates in the future (for instance, the YARN history service seems to have a notion of predicates that can be based on time, user, or other things).


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13398770
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -290,8 +217,88 @@ object HistoryServer {
         }
       }
     
    +  private def parse(args: List[String]): Unit = {
    +    args match {
    +      case ("--dir" | "-d") :: value :: tail =>
    +        set("fs.logDirectory",  value)
    +        parse(tail)
    +
    +      case ("--port" | "-p") :: value :: tail =>
    +        set("ui.port", value)
    +        parse(tail)
    +
    +      case ("-D") :: opt :: value :: tail =>
    +        set(opt, value)
    +        parse(tail)
    +
    +      case ("--help" | "-h") :: tail =>
    +        printUsageAndExit(0)
    +
    +      case Nil =>
    +
    +      case _ =>
    +        printUsageAndExit(1)
    +    }
    +  }
    +
    +  private def set(name: String, value: String) = {
    +    conf.set("spark.history." + name, value)
    +  }
    +
    +  private def printUsageAndExit(exitCode: Int) {
    +    System.err.println(
    +      """
    +      |Usage: HistoryServer [options]
    +      |
    +      |Options are set by passing "-D option value" command line arguments to the class.
    +      |Command line options will override the Spark configuration file and system properties.
    +      |History Server options are always available; additional options depend on the provider.
    +      |
    +      |History Server options:
    +      |
    +      |  ui.port           Port where server will listen for connections (default 18080)
    +      |  ui.acls.enable    Whether to enable view acls for all applications (default false)
    +      |  provider          Name of history provider class (defaults to file system-based provider)
    +      |
    +      |FsHistoryProvider options:
    +      |
    +      |  fs.logDirectory   Directory where app logs are stored (required)
    +      |  fs.updateInterval How often to reload log data from storage (seconds, default 10)
    +      |""".stripMargin)
    +    System.exit(exitCode)
    +  }
    +
     }
     
    +private[spark] abstract class ApplicationHistoryProvider {
    +
    +  /**
    +   * This method should return a list of applications available for the history server to
    +   * show. The listing is assumed to be in descending time order.
    +   *
    +   * An adjusted offset should be returned if the app list has changed and the request
    +   * references an invalid start offset. Otherwise, the provided offset should be returned.
    +   *
    +   * @param offset Starting offset for returned objects.
    +   * @param count Max number of objects to return.
    +   * @return 3-tuple (requested app list, adjusted offset, count of all available apps)
    +   */
    +  def getListing(offset: Int, count: Int): (Seq[ApplicationHistoryInfo], Int, Int)
    --- End diff --
    
    The interface is the right place for the offset / count arguments, because it frees the backend to optimize things in a way that makes sense for it. Making the interface "always return the complete list" means the backend always has to load all the data into memory.
    
    Imagine if the backend is not a bunch of files, but some external server (e.g. the Yarn app timeline server). Returning the whole list means transferring a lot of data from the remote server on every user request, which is very wasteful.
    
    The fact that the FS backend currently keeps the whole list in memory is an implementation detail that can be fixed.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13667375
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,210 @@
    +/*
    + * 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.io.FileNotFoundException
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval",
    +    conf.getInt("spark.history.updateInterval", 10)) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  @volatile private var appList: Seq[ApplicationHistoryInfo] = Nil
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTimeMs()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing() = appList
    +
    +  override def getAppUI(appId: String): SparkUI = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)._2
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  override def getConfig(): Map[String, String] =
    +    Map(("Event Log Location" -> logDir))
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, by not reading
    +   * applications that haven't been updated since last time the logs were checked.
    +   */
    +  private def checkForLogs() = {
    +    lastLogCheckTimeMs = getMonotonicTimeMs()
    +    logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs))
    +    try {
    +      val logStatus = fs.listStatus(new Path(logDir))
    +      val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
    +      val logInfos = logDirs
    +        .filter {
    +            dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))
    +          }
    +
    +      val currentApps = Map[String, ApplicationHistoryInfo](
    +        appList.map(app => (app.id -> app)):_*)
    +
    +      // For any application that either (i) is not listed or (ii) has changed since the last time
    +      // the listing was created (defined by the log dir's modification time), load the app's info.
    +      // Otherwise just reuse what's already in memory.
    +      val newApps = new mutable.ArrayBuffer[ApplicationHistoryInfo](logInfos.size)
    +      for (dir <- logInfos) {
    +        val curr = currentApps.getOrElse(dir.getPath().getName(), null)
    +        if (curr == null || curr.lastUpdated < getModificationTime(dir)) {
    +          try {
    +            newApps += loadAppInfo(dir, false)._1
    +          } catch {
    +            case e: Exception => logError(s"Failed to load app info from directory $dir.")
    +          }
    +        } else {
    +          newApps += curr
    +        }
    +      }
    +
    +      appList = newApps.sortBy { info => -info.endTime }
    +    } catch {
    +      case t: Throwable => logError("Exception in checking for event log updates", t)
    +    }
    +  }
    +
    +  /**
    +   * Parse the application's logs to find out the information we need to build the
    +   * listing page.
    +   *
    +   * When creating the listing of available apps, there is no need to load the whole UI for the
    +   * application. The UI is requested by the HistoryServer (by calling getAppInfo()) when the user
    +   * clicks on a specific application.
    +   *
    +   * @param logDir Directory with application's log files.
    +   * @param renderUI Whether to create the SparkUI for the application.
    +   * @return A 2-tuple `(app info, ui)`. `ui` will be null if `renderUI` is false.
    --- End diff --
    
    I though about that during our discussion, but that would require propagating the ACLs to the other method too; so now we'd be returning a 3-tuple? That would make the ugliness even worse...
    
    (I could restore the ACL field in the info object, but then we'd be wasting a lot of space in memory just to avoid a null check.)


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-46596891
  
    Looks like other PRs are running into the same test issue, so definitely not an issue with my code.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13631207
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -38,56 +39,70 @@ import org.apache.spark.util.Utils
      * 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,
    +    provider: ApplicationHistoryProvider,
         securityManager: SecurityManager,
    -    conf: SparkConf)
    -  extends WebUI(securityManager, HistoryServer.WEB_UI_PORT, conf) with Logging {
    +    port: Int)
    +  extends WebUI(securityManager, port, conf) with Logging {
     
    -  import HistoryServer._
    +  // How many applications to retain
    +  private val retainedApplications = conf.getInt("spark.history.retainedApplications", 50)
     
    -  private val fileSystem = Utils.getHadoopFileSystem(baseLogDir)
       private val localHost = Utils.localHostName()
    --- End diff --
    
    Oops, now this is not 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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13668043
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,210 @@
    +/*
    + * 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.io.FileNotFoundException
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval",
    +    conf.getInt("spark.history.updateInterval", 10)) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  @volatile private var appList: Seq[ApplicationHistoryInfo] = Nil
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTimeMs()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing() = appList
    +
    +  override def getAppUI(appId: String): SparkUI = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)._2
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  override def getConfig(): Map[String, String] =
    +    Map(("Event Log Location" -> logDir))
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, by not reading
    +   * applications that haven't been updated since last time the logs were checked.
    +   */
    +  private def checkForLogs() = {
    +    lastLogCheckTimeMs = getMonotonicTimeMs()
    +    logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs))
    +    try {
    +      val logStatus = fs.listStatus(new Path(logDir))
    +      val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
    +      val logInfos = logDirs
    +        .filter {
    +            dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))
    +          }
    +
    +      val currentApps = Map[String, ApplicationHistoryInfo](
    +        appList.map(app => (app.id -> app)):_*)
    +
    +      // For any application that either (i) is not listed or (ii) has changed since the last time
    +      // the listing was created (defined by the log dir's modification time), load the app's info.
    +      // Otherwise just reuse what's already in memory.
    +      val newApps = new mutable.ArrayBuffer[ApplicationHistoryInfo](logInfos.size)
    +      for (dir <- logInfos) {
    +        val curr = currentApps.getOrElse(dir.getPath().getName(), null)
    +        if (curr == null || curr.lastUpdated < getModificationTime(dir)) {
    +          try {
    +            newApps += loadAppInfo(dir, false)._1
    +          } catch {
    +            case e: Exception => logError(s"Failed to load app info from directory $dir.")
    +          }
    +        } else {
    +          newApps += curr
    +        }
    +      }
    +
    +      appList = newApps.sortBy { info => -info.endTime }
    +    } catch {
    +      case t: Throwable => logError("Exception in checking for event log updates", t)
    +    }
    +  }
    +
    +  /**
    +   * Parse the application's logs to find out the information we need to build the
    +   * listing page.
    +   *
    +   * When creating the listing of available apps, there is no need to load the whole UI for the
    +   * application. The UI is requested by the HistoryServer (by calling getAppInfo()) when the user
    +   * clicks on a specific application.
    +   *
    +   * @param logDir Directory with application's log files.
    +   * @param renderUI Whether to create the SparkUI for the application.
    +   * @return A 2-tuple `(app info, ui)`. `ui` will be null if `renderUI` is false.
    --- End diff --
    
    Ah, sorry, misunderstood what you meant. But in any case, that won't work, because we need to set up the ACLs after the replay happens in L178.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13565555
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,193 @@
    +/*
    + * 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.io.FileNotFoundException
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval", 10) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  @volatile private var appList: Seq[ApplicationHistoryInfo] = Nil
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTime()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing() = appList
    +
    +  override def getAppInfo(appId: String): ApplicationHistoryInfo = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, by not reading
    +   * applications that haven't been updated since last time the logs were checked.
    +   */
    +  def checkForLogs() = {
    --- End diff --
    
    This can 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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13572723
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,193 @@
    +/*
    + * 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.io.FileNotFoundException
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval", 10) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  @volatile private var appList: Seq[ApplicationHistoryInfo] = Nil
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTime()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing() = appList
    +
    +  override def getAppInfo(appId: String): ApplicationHistoryInfo = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, by not reading
    +   * applications that haven't been updated since last time the logs were checked.
    +   */
    +  def checkForLogs() = {
    +    lastLogCheckTimeMs = getMonotonicTime()
    +    logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs))
    +    try {
    +      val logStatus = fs.listStatus(new Path(logDir))
    +      val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
    +      val logInfos = logDirs
    +        .filter {
    +            dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))
    +          }
    +
    +      val currentApps = Map[String, ApplicationHistoryInfo](
    +        appList.map(app => (app.id -> app)):_*)
    +
    +      // For any application that either (i) is not listed or (ii) has changed since the last time
    +      // the listing was created (defined by the log dir's modification time), load the app's info.
    +      // Otherwise just reuse what's already in memory.
    +      val newApps = new mutable.ArrayBuffer[ApplicationHistoryInfo](logInfos.size)
    +      for (dir <- logInfos) {
    +        val curr = currentApps.getOrElse(dir.getPath().getName(), null)
    +        if (curr == null || curr.lastUpdated < getModificationTime(dir)) {
    +          try {
    +            newApps += loadAppInfo(dir, false)
    --- End diff --
    
    So would it be a problem if the user goes ahead and clicks on all of the applications? Looks like we still need `spark.history.retainedApplications` because of that. Though it seems that this value now represents the cache size for rendered SparkUI, rather than the number of SparkUIs that can be displayed to the user.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-46767513
  
    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-1768] History server enhancements.

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

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


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-46373778
  
    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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13401102
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -290,8 +217,88 @@ object HistoryServer {
         }
       }
     
    +  private def parse(args: List[String]): Unit = {
    +    args match {
    +      case ("--dir" | "-d") :: value :: tail =>
    +        set("fs.logDirectory",  value)
    +        parse(tail)
    +
    +      case ("--port" | "-p") :: value :: tail =>
    +        set("ui.port", value)
    +        parse(tail)
    +
    +      case ("-D") :: opt :: value :: tail =>
    +        set(opt, value)
    +        parse(tail)
    +
    +      case ("--help" | "-h") :: tail =>
    +        printUsageAndExit(0)
    +
    +      case Nil =>
    +
    +      case _ =>
    +        printUsageAndExit(1)
    +    }
    +  }
    +
    +  private def set(name: String, value: String) = {
    +    conf.set("spark.history." + name, value)
    +  }
    +
    +  private def printUsageAndExit(exitCode: Int) {
    +    System.err.println(
    +      """
    +      |Usage: HistoryServer [options]
    +      |
    +      |Options are set by passing "-D option value" command line arguments to the class.
    +      |Command line options will override the Spark configuration file and system properties.
    +      |History Server options are always available; additional options depend on the provider.
    +      |
    +      |History Server options:
    +      |
    +      |  ui.port           Port where server will listen for connections (default 18080)
    +      |  ui.acls.enable    Whether to enable view acls for all applications (default false)
    +      |  provider          Name of history provider class (defaults to file system-based provider)
    +      |
    +      |FsHistoryProvider options:
    +      |
    +      |  fs.logDirectory   Directory where app logs are stored (required)
    +      |  fs.updateInterval How often to reload log data from storage (seconds, default 10)
    +      |""".stripMargin)
    +    System.exit(exitCode)
    +  }
    +
     }
     
    +private[spark] abstract class ApplicationHistoryProvider {
    +
    +  /**
    +   * This method should return a list of applications available for the history server to
    +   * show. The listing is assumed to be in descending time order.
    +   *
    +   * An adjusted offset should be returned if the app list has changed and the request
    +   * references an invalid start offset. Otherwise, the provided offset should be returned.
    +   *
    +   * @param offset Starting offset for returned objects.
    +   * @param count Max number of objects to return.
    +   * @return 3-tuple (requested app list, adjusted offset, count of all available apps)
    +   */
    +  def getListing(offset: Int, count: Int): (Seq[ApplicationHistoryInfo], Int, Int)
    --- End diff --
    
    If we did want to optimize this, wouldn't the way be to have one function that returns a listing of applications and another function that returns the full in-memory representation of a given application? The nit wouldn't restrict the optimizations to be tied to sequential access (in fact, the one existing optimization we have is based on caching and not tied to sequential access at all).
    
    The offset is also a bit weird to have here because it implies an ordering, but there isn't any discussion of how the ordering is defined and whether it should be stable over time.
    
    Anyways, why not just keep this simple rather than make it more clunky for some optimization we didn't even write yet?


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13572562
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,193 @@
    +/*
    + * 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.io.FileNotFoundException
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval", 10) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  @volatile private var appList: Seq[ApplicationHistoryInfo] = Nil
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTime()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing() = appList
    +
    +  override def getAppInfo(appId: String): ApplicationHistoryInfo = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, by not reading
    +   * applications that haven't been updated since last time the logs were checked.
    +   */
    +  def checkForLogs() = {
    +    lastLogCheckTimeMs = getMonotonicTime()
    +    logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs))
    +    try {
    +      val logStatus = fs.listStatus(new Path(logDir))
    +      val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
    +      val logInfos = logDirs
    +        .filter {
    +            dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))
    +          }
    +
    +      val currentApps = Map[String, ApplicationHistoryInfo](
    +        appList.map(app => (app.id -> app)):_*)
    +
    +      // For any application that either (i) is not listed or (ii) has changed since the last time
    +      // the listing was created (defined by the log dir's modification time), load the app's info.
    +      // Otherwise just reuse what's already in memory.
    +      val newApps = new mutable.ArrayBuffer[ApplicationHistoryInfo](logInfos.size)
    +      for (dir <- logInfos) {
    +        val curr = currentApps.getOrElse(dir.getPath().getName(), null)
    +        if (curr == null || curr.lastUpdated < getModificationTime(dir)) {
    +          try {
    +            newApps += loadAppInfo(dir, false)
    +          } catch {
    +            case e: Exception => logError(s"Failed to load app info from directory $dir.")
    +          }
    +        } else {
    +          newApps += curr
    +        }
    +      }
    +
    +      appList = newApps.sortBy { info => -info.endTime }
    +    } catch {
    +      case t: Throwable => logError("Exception in checking for event log updates", t)
    +    }
    +  }
    +
    +  /**
    +   * Parse the application's logs to find out the information we need to build the
    +   * listing page.
    +   */
    +  private def loadAppInfo(logDir: FileStatus, renderUI: Boolean): ApplicationHistoryInfo = {
    +    val elogInfo = EventLoggingListener.parseLoggingInfo(logDir.getPath(), fs)
    +    val path = logDir.getPath
    +    val appId = path.getName
    +    val replayBus = new ReplayListenerBus(elogInfo.logPaths, fs, elogInfo.compressionCodec)
    +    val appListener = new ApplicationEventListener
    +    replayBus.addListener(appListener)
    +
    +    val ui: SparkUI = if (renderUI) {
    +        val conf = this.conf.clone()
    +        val appSecManager = new SecurityManager(conf)
    +        new SparkUI(conf, appSecManager, replayBus, appId, "/history/" + appId)
    +        // Do not call ui.bind() to avoid creating a new server for each application
    +      } else {
    +        null
    +      }
    +
    +    replayBus.replay()
    +    ApplicationHistoryInfo(appId,
    +      appListener.appName,
    +      appListener.startTime,
    +      appListener.endTime,
    +      getModificationTime(logDir),
    +      appListener.sparkUser,
    +      if (renderUI) appListener.viewAcls else null,
    +      ui)
    +  }
    +
    +  /** Return when this directory was last modified. */
    +  private def getModificationTime(dir: FileStatus): Long = {
    +    try {
    +      val logFiles = fs.listStatus(dir.getPath)
    +      if (logFiles != null && !logFiles.isEmpty) {
    +        logFiles.map(_.getModificationTime).max
    +      } else {
    +        dir.getModificationTime
    +      }
    +    } catch {
    +      case t: Throwable =>
    +        logError("Exception in accessing modification time of %s".format(dir.getPath), t)
    +        -1L
    +    }
    +  }
    +
    +  /** Returns the system's mononotically increasing time. */
    +  private def getMonotonicTime() = System.nanoTime() / (1000 * 1000)
    +
    --- End diff --
    
    That's fair. No need to add a shutdown hook then.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13630671
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala ---
    @@ -45,32 +42,34 @@ private[spark] class HistoryServerArguments(args: Array[String]) {
           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)) {
    -      System.err.println("Logging directory specified does not exist: %s".format(logDir))
    -      printUsageAndExit(1)
    -    }
    -    if (!fileSystem.getFileStatus(path).isDir) {
    -      System.err.println("Logging directory specified is not a directory: %s".format(logDir))
    -      printUsageAndExit(1)
    +    if (logDir != null) {
    +      conf.set("spark.history.fs.logDirectory", logDir)
         }
       }
     
       private def printUsageAndExit(exitCode: Int) {
         System.err.println(
    -      "Usage: HistoryServer [options]\n" +
    -      "\n" +
    -      "Options:\n" +
    -      "  -d DIR,  --dir DIR     Location of event log files")
    +      """
    +      |Usage: HistoryServer
    +      |
    +      |Configuration options can be set by setting the corresponding JVM system property.
    +      |History Server options are always available; additional options depend on the provider.
    +      |
    +      |History Server options:
    +      |
    +      |  spark.history.ui.port      Port where server will listen for connections (default 18080)
    +      |  spark.history.acls.enable  Whether to enable view acls for all applications (default false)
    +      |  spark.history.provider     Name of history provider class (defaults to file system-based
    +      |                             provider)
    +      |  spark.history.retainedApplications Max number of application UIs to keep loaded in memory
    +      |                             (default 50)
    --- End diff --
    
    minor: indentation is off. Could you align the other config descriptions with that of `spark.history.retainedApplications`?


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-46743627
  
    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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-46384763
  
    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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13602973
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,193 @@
    +/*
    + * 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.io.FileNotFoundException
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval", 10) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  @volatile private var appList: Seq[ApplicationHistoryInfo] = Nil
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTime()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing() = appList
    +
    +  override def getAppInfo(appId: String): ApplicationHistoryInfo = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, by not reading
    +   * applications that haven't been updated since last time the logs were checked.
    +   */
    +  def checkForLogs() = {
    +    lastLogCheckTimeMs = getMonotonicTime()
    +    logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs))
    +    try {
    +      val logStatus = fs.listStatus(new Path(logDir))
    +      val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
    +      val logInfos = logDirs
    +        .filter {
    +            dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))
    +          }
    +
    +      val currentApps = Map[String, ApplicationHistoryInfo](
    +        appList.map(app => (app.id -> app)):_*)
    +
    +      // For any application that either (i) is not listed or (ii) has changed since the last time
    +      // the listing was created (defined by the log dir's modification time), load the app's info.
    +      // Otherwise just reuse what's already in memory.
    +      val newApps = new mutable.ArrayBuffer[ApplicationHistoryInfo](logInfos.size)
    +      for (dir <- logInfos) {
    +        val curr = currentApps.getOrElse(dir.getPath().getName(), null)
    +        if (curr == null || curr.lastUpdated < getModificationTime(dir)) {
    +          try {
    +            newApps += loadAppInfo(dir, false)
    --- End diff --
    
    Correct, retainedApplications now control how many UIs are kept in memory; the cache evicts the least recently used one when the max size is exceeded.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-42717198
  
    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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-46742995
  
    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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13603025
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,193 @@
    +/*
    + * 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.io.FileNotFoundException
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval", 10) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  @volatile private var appList: Seq[ApplicationHistoryInfo] = Nil
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTime()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing() = appList
    +
    +  override def getAppInfo(appId: String): ApplicationHistoryInfo = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, by not reading
    +   * applications that haven't been updated since last time the logs were checked.
    +   */
    +  def checkForLogs() = {
    +    lastLogCheckTimeMs = getMonotonicTime()
    +    logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs))
    +    try {
    +      val logStatus = fs.listStatus(new Path(logDir))
    +      val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
    +      val logInfos = logDirs
    +        .filter {
    +            dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))
    +          }
    +
    +      val currentApps = Map[String, ApplicationHistoryInfo](
    +        appList.map(app => (app.id -> app)):_*)
    --- End diff --
    
    Still thinking about it. I'll update the patch once I find a solution that I'm happy with.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13668299
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,210 @@
    +/*
    + * 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.io.FileNotFoundException
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval",
    +    conf.getInt("spark.history.updateInterval", 10)) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  @volatile private var appList: Seq[ApplicationHistoryInfo] = Nil
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTimeMs()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing() = appList
    +
    +  override def getAppUI(appId: String): SparkUI = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)._2
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  override def getConfig(): Map[String, String] =
    +    Map(("Event Log Location" -> logDir))
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, by not reading
    +   * applications that haven't been updated since last time the logs were checked.
    +   */
    +  private def checkForLogs() = {
    +    lastLogCheckTimeMs = getMonotonicTimeMs()
    +    logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs))
    +    try {
    +      val logStatus = fs.listStatus(new Path(logDir))
    +      val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
    +      val logInfos = logDirs
    +        .filter {
    +            dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))
    +          }
    +
    +      val currentApps = Map[String, ApplicationHistoryInfo](
    +        appList.map(app => (app.id -> app)):_*)
    +
    +      // For any application that either (i) is not listed or (ii) has changed since the last time
    +      // the listing was created (defined by the log dir's modification time), load the app's info.
    +      // Otherwise just reuse what's already in memory.
    +      val newApps = new mutable.ArrayBuffer[ApplicationHistoryInfo](logInfos.size)
    +      for (dir <- logInfos) {
    +        val curr = currentApps.getOrElse(dir.getPath().getName(), null)
    +        if (curr == null || curr.lastUpdated < getModificationTime(dir)) {
    +          try {
    +            newApps += loadAppInfo(dir, false)._1
    +          } catch {
    +            case e: Exception => logError(s"Failed to load app info from directory $dir.")
    +          }
    +        } else {
    +          newApps += curr
    +        }
    +      }
    +
    +      appList = newApps.sortBy { info => -info.endTime }
    +    } catch {
    +      case t: Throwable => logError("Exception in checking for event log updates", t)
    +    }
    +  }
    +
    +  /**
    +   * Parse the application's logs to find out the information we need to build the
    +   * listing page.
    +   *
    +   * When creating the listing of available apps, there is no need to load the whole UI for the
    +   * application. The UI is requested by the HistoryServer (by calling getAppInfo()) when the user
    +   * clicks on a specific application.
    +   *
    +   * @param logDir Directory with application's log files.
    +   * @param renderUI Whether to create the SparkUI for the application.
    +   * @return A 2-tuple `(app info, ui)`. `ui` will be null if `renderUI` is false.
    --- End diff --
    
    Oh I see, because you need to replay to get the ACLs. Then it's fine.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13367311
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -290,8 +217,88 @@ object HistoryServer {
         }
       }
     
    +  private def parse(args: List[String]): Unit = {
    +    args match {
    +      case ("--dir" | "-d") :: value :: tail =>
    +        set("fs.logDirectory",  value)
    +        parse(tail)
    +
    +      case ("--port" | "-p") :: value :: tail =>
    +        set("ui.port", value)
    +        parse(tail)
    +
    +      case ("-D") :: opt :: value :: tail =>
    +        set(opt, value)
    +        parse(tail)
    +
    +      case ("--help" | "-h") :: tail =>
    +        printUsageAndExit(0)
    +
    +      case Nil =>
    +
    +      case _ =>
    +        printUsageAndExit(1)
    +    }
    +  }
    +
    +  private def set(name: String, value: String) = {
    +    conf.set("spark.history." + name, value)
    +  }
    +
    +  private def printUsageAndExit(exitCode: Int) {
    +    System.err.println(
    +      """
    +      |Usage: HistoryServer [options]
    +      |
    +      |Options are set by passing "-D option value" command line arguments to the class.
    +      |Command line options will override the Spark configuration file and system properties.
    +      |History Server options are always available; additional options depend on the provider.
    +      |
    +      |History Server options:
    +      |
    +      |  ui.port           Port where server will listen for connections (default 18080)
    +      |  ui.acls.enable    Whether to enable view acls for all applications (default false)
    +      |  provider          Name of history provider class (defaults to file system-based provider)
    +      |
    +      |FsHistoryProvider options:
    +      |
    +      |  fs.logDirectory   Directory where app logs are stored (required)
    +      |  fs.updateInterval How often to reload log data from storage (seconds, default 10)
    +      |""".stripMargin)
    +    System.exit(exitCode)
    +  }
    +
     }
     
    +private[spark] abstract class ApplicationHistoryProvider {
    +
    +  /**
    +   * This method should return a list of applications available for the history server to
    +   * show. The listing is assumed to be in descending time order.
    +   *
    +   * An adjusted offset should be returned if the app list has changed and the request
    +   * references an invalid start offset. Otherwise, the provided offset should be returned.
    +   *
    +   * @param offset Starting offset for returned objects.
    +   * @param count Max number of objects to return.
    +   * @return 3-tuple (requested app list, adjusted offset, count of all available apps)
    +   */
    +  def getListing(offset: Int, count: Int): (Seq[ApplicationHistoryInfo], Int, Int)
    --- End diff --
    
    Having a notion of offsets in this interface is a bit awkward. Why not have this just return all applications and then have the caller deal with offsets.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13572856
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,193 @@
    +/*
    + * 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.io.FileNotFoundException
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval", 10) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  @volatile private var appList: Seq[ApplicationHistoryInfo] = Nil
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTime()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing() = appList
    +
    +  override def getAppInfo(appId: String): ApplicationHistoryInfo = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, by not reading
    +   * applications that haven't been updated since last time the logs were checked.
    +   */
    +  def checkForLogs() = {
    +    lastLogCheckTimeMs = getMonotonicTime()
    +    logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs))
    +    try {
    +      val logStatus = fs.listStatus(new Path(logDir))
    +      val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
    +      val logInfos = logDirs
    +        .filter {
    +            dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))
    +          }
    +
    +      val currentApps = Map[String, ApplicationHistoryInfo](
    +        appList.map(app => (app.id -> app)):_*)
    --- End diff --
    
    Yes, but here we unroll it into a map anyway, so it will still OOM.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13398426
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -290,8 +217,88 @@ object HistoryServer {
         }
       }
     
    +  private def parse(args: List[String]): Unit = {
    +    args match {
    +      case ("--dir" | "-d") :: value :: tail =>
    +        set("fs.logDirectory",  value)
    +        parse(tail)
    +
    +      case ("--port" | "-p") :: value :: tail =>
    +        set("ui.port", value)
    +        parse(tail)
    +
    +      case ("-D") :: opt :: value :: tail =>
    +        set(opt, value)
    +        parse(tail)
    +
    +      case ("--help" | "-h") :: tail =>
    +        printUsageAndExit(0)
    +
    +      case Nil =>
    +
    +      case _ =>
    +        printUsageAndExit(1)
    +    }
    +  }
    +
    +  private def set(name: String, value: String) = {
    +    conf.set("spark.history." + name, value)
    +  }
    +
    +  private def printUsageAndExit(exitCode: Int) {
    +    System.err.println(
    +      """
    +      |Usage: HistoryServer [options]
    +      |
    +      |Options are set by passing "-D option value" command line arguments to the class.
    +      |Command line options will override the Spark configuration file and system properties.
    --- End diff --
    
    Yes, true. I'll drop the config comment for now.
    
    But in general I think it's a good idea for the HS to read configuration from a file. Either loading the default config file used by spark-submit, or a config file defined as a command line argument. I assume the most common use case is for someone to set this up as something that runs as a system service (e.g. through init), so separating the execution (i.e. the script in /etc/init.d) from configuration would be a nice enhancement.
    
    Can be done separately 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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-46340525
  
    We should exclude the entire org.apache.spark.deploy package. @vanzin checkout `project/MimaExcludes.scala` and add `excludePackage` for 1.1


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-46382728
  
     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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13398162
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,203 @@
    +/*
    + * 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.io.FileNotFoundException
    +import java.util.concurrent.atomic.AtomicReference
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval", 10) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  private val appList = new AtomicReference[Seq[ApplicationHistoryInfo]](Nil)
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTime()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing(offset: Int, count: Int) = {
    +    val list = appList.get()
    +    val theOffset = if (offset < list.size) offset else 0
    +    (list.slice(theOffset, Math.min(theOffset + count, list.size)), theOffset, list.size)
    +  }
    +
    +  override def getAppInfo(appId: String): ApplicationHistoryInfo = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, but not reading
    +   * applications that hasn't been updated since last time the logs were checked.
    +   */
    +  def checkForLogs() = synchronized {
    +    lastLogCheckTimeMs = getMonotonicTime()
    +    logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs))
    +    try {
    +      val logStatus = fs.listStatus(new Path(logDir))
    +      val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
    +      val logInfos = logDirs
    +        .filter {
    +            dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))
    +          }
    +
    +      var currentApps = Map[String, ApplicationHistoryInfo](
    +        appList.get().map(app => (app.id -> app)):_*)
    +
    +      // For any application that either (i) is not listed or (ii) has changed since the last time
    +      // the listing was created (defined by the log dir's modification time), load the app's info.
    +      // Otherwise just reuse what's already in memory.
    +      val newApps = new mutable.ArrayBuffer[ApplicationHistoryInfo](logInfos.size)
    +      for (dir <- logInfos) {
    +        val curr = currentApps.getOrElse(dir.getPath().getName(), null)
    +        if (curr == null || curr.lastUpdated < getModificationTime(dir)) {
    +          try {
    +            newApps += loadAppInfo(dir, false)
    +          } catch {
    +            case e: Exception => logError(s"Failed to load app info from directory $dir.")
    +          }
    +        } else {
    +          newApps += curr
    +        }
    +      }
    +
    +      appList.set(newApps.sortBy { info => -info.lastUpdated })
    +    } catch {
    +      case t: Throwable => logError("Exception in checking for event log updates", t)
    +    }
    +  }
    +
    +  /**
    +   * Parse the application's logs to find out the information we need to build the
    +   * listing page.
    +   */
    +  private def loadAppInfo(logDir: FileStatus, renderUI: Boolean): ApplicationHistoryInfo = {
    +    val elogInfo = EventLoggingListener.parseLoggingInfo(logDir.getPath(), fs)
    +    val path = logDir.getPath
    +    val appId = path.getName
    +    val replayBus = new ReplayListenerBus(elogInfo.logPaths, fs, elogInfo.compressionCodec)
    +    val appListener = new ApplicationEventListener
    +    replayBus.addListener(appListener)
    +
    +    val ui: SparkUI = if (renderUI) {
    +        val conf = this.conf.clone()
    +        val appSecManager = new SecurityManager(conf)
    +        new SparkUI(conf, appSecManager, replayBus, appId, "/history/" + appId)
    +        // Do not call ui.bind() to avoid creating a new server for each application
    +      } else {
    +        null
    +      }
    +
    +    replayBus.replay()
    +    val appName = appListener.appName
    +    val sparkUser = appListener.sparkUser
    +    val startTime = appListener.startTime
    +    val endTime = appListener.endTime
    +    val lastUpdated = getModificationTime(logDir)
    +    ApplicationHistoryInfo(appId,
    +      appListener.appName,
    +      appListener.startTime,
    +      appListener.endTime,
    +      getModificationTime(logDir),
    +      appListener.sparkUser,
    +      if (renderUI) appListener.viewAcls else null,
    +      ui)
    +  }
    +
    +  /** Return when this directory was last modified. */
    +  private def getModificationTime(dir: FileStatus): Long = {
    +    try {
    +      val logFiles = fs.listStatus(dir.getPath)
    +      if (logFiles != null && !logFiles.isEmpty) {
    +        logFiles.map(_.getModificationTime).max
    +      } else {
    +        dir.getModificationTime
    +      }
    +    } catch {
    +      case t: Throwable =>
    +        logError("Exception in accessing modification time of %s".format(dir.getPath), t)
    +        -1L
    +    }
    +  }
    +
    +  /** Returns the system's mononotically increasing time. */
    +  private def getMonotonicTime() = System.nanoTime() / (1000 * 1000)
    --- End diff --
    
    System.currentTimeMillis() is not monotonic.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13619981
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala ---
    @@ -67,10 +69,29 @@ private[spark] class HistoryServerArguments(args: Array[String]) {
     
       private def printUsageAndExit(exitCode: Int) {
         System.err.println(
    -      "Usage: HistoryServer [options]\n" +
    -      "\n" +
    -      "Options:\n" +
    -      "  -d DIR,  --dir DIR     Location of event log files")
    +      """
    +      |Usage: HistoryServer [-d logDir]
    +      |
    +      |The preferred way to pass options is to set the configuration below using
    +      |SPARK_HISTORY_OPTS. The "-d" command line argument is avalable for backwards
    --- End diff --
    
    avalable -> available


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-46767578
  
     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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13565390
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -38,56 +39,75 @@ import org.apache.spark.util.Utils
      * 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,
    +    provider: ApplicationHistoryProvider,
         securityManager: SecurityManager,
    -    conf: SparkConf)
    -  extends WebUI(securityManager, HistoryServer.WEB_UI_PORT, conf) with Logging {
    +    port: Int)
    +  extends WebUI(securityManager, port, conf) with Logging {
    +
    +  // How many applications to retain
    +  private val retainedApplications = conf.getInt("spark.history.retainedApplications", 50)
     
    -  import HistoryServer._
    +  // set whether to enable or disable view acls for all applications
    +  private val uiAclsEnabled = conf.getBoolean("spark.history.ui.acls.enable", false)
     
    -  private val fileSystem = Utils.getHadoopFileSystem(baseLogDir)
       private val localHost = Utils.localHostName()
       private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHost)
     
    -  // A timestamp of when the disk was last accessed to check for log updates
    -  private var lastLogCheckTime = -1L
    +  private val appLoader = new CacheLoader[String, SparkUI] {
    +    override def load(key: String): SparkUI = {
    +      val info = provider.getAppInfo(key)
    +      if (info == null) {
    +        throw new NoSuchElementException()
    +      }
    +      info.ui.getSecurityManager.setUIAcls(uiAclsEnabled)
    +      info.ui.getSecurityManager.setViewAcls(info.sparkUser, info.viewAcls)
    +      attachSparkUI(info.ui)
    +      info.ui
    +    }
    +  }
     
    -  // Number of completed applications found in this directory
    -  private var numCompletedApplications = 0
    +  private val appCache = CacheBuilder.newBuilder()
    +    .maximumSize(retainedApplications)
    +    .removalListener(new RemovalListener[String, SparkUI] {
    +      override def onRemoval(rm: RemovalNotification[String, SparkUI]) = {
    +        detachSparkUI(rm.getValue())
    +      }
    +    })
    +    .build(appLoader)
    +
    +  private val loaderServlet = new HttpServlet {
    +    protected override def doGet(req: HttpServletRequest, res: HttpServletResponse): Unit = {
    +      val parts = req.getPathInfo().split("/")
    +      if (parts.length < 2) {
    +        res.setStatus(HttpServletResponse.SC_BAD_REQUEST)
    --- End diff --
    
    I would also log a warning, possibly with the request's URI. Debugging why Jetty isn't giving the expected responses can be a painful process.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13572389
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -290,8 +212,81 @@ object HistoryServer {
         }
       }
     
    +  private def parse(args: List[String]): Unit = {
    +    args match {
    +      case ("--dir" | "-d") :: value :: tail =>
    +        set("fs.logDirectory",  value)
    +        parse(tail)
    +
    +      case ("-D") :: opt :: value :: tail =>
    --- End diff --
    
    Hacky I agree, but it would be good to keep it consistent with other spark daemons (i.e. standalone master / worker), at least for 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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-46377452
  
    Test failure in latest build seems unrelated to my changes?


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#issuecomment-46742965
  
    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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13570813
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,193 @@
    +/*
    + * 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.io.FileNotFoundException
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval", 10) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  @volatile private var appList: Seq[ApplicationHistoryInfo] = Nil
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTime()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing() = appList
    +
    +  override def getAppInfo(appId: String): ApplicationHistoryInfo = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, by not reading
    +   * applications that haven't been updated since last time the logs were checked.
    +   */
    +  def checkForLogs() = {
    +    lastLogCheckTimeMs = getMonotonicTime()
    +    logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs))
    +    try {
    +      val logStatus = fs.listStatus(new Path(logDir))
    +      val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
    +      val logInfos = logDirs
    +        .filter {
    +            dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))
    +          }
    +
    +      val currentApps = Map[String, ApplicationHistoryInfo](
    +        appList.map(app => (app.id -> app)):_*)
    --- End diff --
    
    Yeah, I'm still thinking about the best way to get rid of this map. I'm not a fan of maps for storing large amounts of entries because they tend to waste a lot of 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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13565544
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,193 @@
    +/*
    + * 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.io.FileNotFoundException
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval", 10) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  @volatile private var appList: Seq[ApplicationHistoryInfo] = Nil
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTime()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing() = appList
    +
    +  override def getAppInfo(appId: String): ApplicationHistoryInfo = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, by not reading
    +   * applications that haven't been updated since last time the logs were checked.
    +   */
    +  def checkForLogs() = {
    +    lastLogCheckTimeMs = getMonotonicTime()
    +    logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs))
    +    try {
    +      val logStatus = fs.listStatus(new Path(logDir))
    +      val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
    +      val logInfos = logDirs
    +        .filter {
    +            dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))
    +          }
    +
    +      val currentApps = Map[String, ApplicationHistoryInfo](
    +        appList.map(app => (app.id -> app)):_*)
    +
    +      // For any application that either (i) is not listed or (ii) has changed since the last time
    +      // the listing was created (defined by the log dir's modification time), load the app's info.
    +      // Otherwise just reuse what's already in memory.
    +      val newApps = new mutable.ArrayBuffer[ApplicationHistoryInfo](logInfos.size)
    +      for (dir <- logInfos) {
    +        val curr = currentApps.getOrElse(dir.getPath().getName(), null)
    +        if (curr == null || curr.lastUpdated < getModificationTime(dir)) {
    +          try {
    +            newApps += loadAppInfo(dir, false)
    +          } catch {
    +            case e: Exception => logError(s"Failed to load app info from directory $dir.")
    +          }
    +        } else {
    +          newApps += curr
    +        }
    +      }
    +
    +      appList = newApps.sortBy { info => -info.endTime }
    +    } catch {
    +      case t: Throwable => logError("Exception in checking for event log updates", t)
    +    }
    +  }
    +
    +  /**
    +   * Parse the application's logs to find out the information we need to build the
    +   * listing page.
    +   */
    +  private def loadAppInfo(logDir: FileStatus, renderUI: Boolean): ApplicationHistoryInfo = {
    +    val elogInfo = EventLoggingListener.parseLoggingInfo(logDir.getPath(), fs)
    +    val path = logDir.getPath
    +    val appId = path.getName
    +    val replayBus = new ReplayListenerBus(elogInfo.logPaths, fs, elogInfo.compressionCodec)
    +    val appListener = new ApplicationEventListener
    +    replayBus.addListener(appListener)
    +
    +    val ui: SparkUI = if (renderUI) {
    +        val conf = this.conf.clone()
    +        val appSecManager = new SecurityManager(conf)
    +        new SparkUI(conf, appSecManager, replayBus, appId, "/history/" + appId)
    +        // Do not call ui.bind() to avoid creating a new server for each application
    +      } else {
    +        null
    +      }
    +
    +    replayBus.replay()
    +    ApplicationHistoryInfo(appId,
    +      appListener.appName,
    +      appListener.startTime,
    +      appListener.endTime,
    +      getModificationTime(logDir),
    +      appListener.sparkUser,
    +      if (renderUI) appListener.viewAcls else null,
    +      ui)
    +  }
    +
    +  /** Return when this directory was last modified. */
    +  private def getModificationTime(dir: FileStatus): Long = {
    +    try {
    +      val logFiles = fs.listStatus(dir.getPath)
    +      if (logFiles != null && !logFiles.isEmpty) {
    +        logFiles.map(_.getModificationTime).max
    +      } else {
    +        dir.getModificationTime
    +      }
    +    } catch {
    +      case t: Throwable =>
    +        logError("Exception in accessing modification time of %s".format(dir.getPath), t)
    +        -1L
    +    }
    +  }
    +
    +  /** Returns the system's mononotically increasing time. */
    +  private def getMonotonicTime() = System.nanoTime() / (1000 * 1000)
    +
    --- End diff --
    
    This is missing a `def stop()`, which should kill the background log checking thread.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13630483
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,214 @@
    +/*
    + * 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.io.FileNotFoundException
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval",
    +    conf.getInt("spark.history.updateInterval", 10)) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory", null)
    +  if (logDir == null) {
    +    throw new IllegalArgumentException("Logging directory must be specified.")
    +  }
    +
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  @volatile private var appList: Seq[ApplicationHistoryInfo] = Nil
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTimeMs()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing() = appList
    +
    +  override def getAppUI(appId: String): SparkUI = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)._2
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  override def getConfig(): Map[String, String] =
    +    Map(("Event Log Location" -> logDir))
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, by not reading
    +   * applications that haven't been updated since last time the logs were checked.
    +   */
    +  private def checkForLogs() = {
    +    lastLogCheckTimeMs = getMonotonicTimeMs()
    +    logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs))
    +    try {
    +      val logStatus = fs.listStatus(new Path(logDir))
    +      val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
    +      val logInfos = logDirs
    +        .filter {
    +            dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))
    +          }
    --- End diff --
    
    minor: formatting is off:
    ```
    val logInfos = logDirs.filter { dir =>
      fs.isFile(...)
    }
    ```


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13618615
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala ---
    @@ -67,10 +69,29 @@ private[spark] class HistoryServerArguments(args: Array[String]) {
     
       private def printUsageAndExit(exitCode: Int) {
         System.err.println(
    -      "Usage: HistoryServer [options]\n" +
    -      "\n" +
    -      "Options:\n" +
    -      "  -d DIR,  --dir DIR     Location of event log files")
    +      """
    +      |Usage: HistoryServer [-d logDir]
    +      |
    +      |The preferred way to pass options is to set the configuration below using
    +      |SPARK_HISTORY_OPTS. The "-d" command line argument is avalable for backwards
    +      |compatibility, and overrides "spark.history.fs.logDirectory".
    +      |
    +      |History Server options are always available; additional options depend on the provider.
    +      |
    +      |History Server options:
    +      |
    +      |  spark.history.ui.port      Port where server will listen for connections (default 18080)
    +      |  spark.history.acls.enable  Whether to enable view acls for all applications (default false)
    +      |  spark.history.provider     Name of history provider class (defaults to file system-based
    +      |                             provider)
    +      |
    +      |FsHistoryProvider options:
    +      |
    +      |  spark.history.fs.logDirectory    Directory where app logs are stored (required)
    --- End diff --
    
    That is mentioned above. I'd like to keep it clear that the config variables approach is preferable, since "-d" won't make sense when the provider class is not the fs-based one.


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13365826
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,203 @@
    +/*
    + * 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.io.FileNotFoundException
    +import java.util.concurrent.atomic.AtomicReference
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval", 10) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory")
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  private val appList = new AtomicReference[Seq[ApplicationHistoryInfo]](Nil)
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTime()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing(offset: Int, count: Int) = {
    +    val list = appList.get()
    +    val theOffset = if (offset < list.size) offset else 0
    +    (list.slice(theOffset, Math.min(theOffset + count, list.size)), theOffset, list.size)
    +  }
    +
    +  override def getAppInfo(appId: String): ApplicationHistoryInfo = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, but not reading
    +   * applications that hasn't been updated since last time the logs were checked.
    +   */
    +  def checkForLogs() = synchronized {
    +    lastLogCheckTimeMs = getMonotonicTime()
    +    logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs))
    +    try {
    +      val logStatus = fs.listStatus(new Path(logDir))
    +      val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
    +      val logInfos = logDirs
    +        .filter {
    +            dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))
    +          }
    +
    +      var currentApps = Map[String, ApplicationHistoryInfo](
    +        appList.get().map(app => (app.id -> app)):_*)
    +
    +      // For any application that either (i) is not listed or (ii) has changed since the last time
    +      // the listing was created (defined by the log dir's modification time), load the app's info.
    +      // Otherwise just reuse what's already in memory.
    +      val newApps = new mutable.ArrayBuffer[ApplicationHistoryInfo](logInfos.size)
    +      for (dir <- logInfos) {
    +        val curr = currentApps.getOrElse(dir.getPath().getName(), null)
    +        if (curr == null || curr.lastUpdated < getModificationTime(dir)) {
    +          try {
    +            newApps += loadAppInfo(dir, false)
    +          } catch {
    +            case e: Exception => logError(s"Failed to load app info from directory $dir.")
    +          }
    +        } else {
    +          newApps += curr
    +        }
    +      }
    +
    +      appList.set(newApps.sortBy { info => -info.lastUpdated })
    +    } catch {
    +      case t: Throwable => logError("Exception in checking for event log updates", t)
    +    }
    +  }
    +
    +  /**
    +   * Parse the application's logs to find out the information we need to build the
    +   * listing page.
    +   */
    +  private def loadAppInfo(logDir: FileStatus, renderUI: Boolean): ApplicationHistoryInfo = {
    +    val elogInfo = EventLoggingListener.parseLoggingInfo(logDir.getPath(), fs)
    +    val path = logDir.getPath
    +    val appId = path.getName
    +    val replayBus = new ReplayListenerBus(elogInfo.logPaths, fs, elogInfo.compressionCodec)
    +    val appListener = new ApplicationEventListener
    +    replayBus.addListener(appListener)
    +
    +    val ui: SparkUI = if (renderUI) {
    +        val conf = this.conf.clone()
    +        val appSecManager = new SecurityManager(conf)
    +        new SparkUI(conf, appSecManager, replayBus, appId, "/history/" + appId)
    +        // Do not call ui.bind() to avoid creating a new server for each application
    +      } else {
    +        null
    +      }
    +
    +    replayBus.replay()
    +    val appName = appListener.appName
    --- End diff --
    
    Here you create 5 variables that are never 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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13630509
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -0,0 +1,214 @@
    +/*
    + * 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.io.FileNotFoundException
    +
    +import scala.collection.mutable
    +
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +import org.apache.spark.scheduler._
    +import org.apache.spark.ui.SparkUI
    +import org.apache.spark.util.Utils
    +
    +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider
    +  with Logging {
    +
    +  // Interval between each check for event log updates
    +  private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval",
    +    conf.getInt("spark.history.updateInterval", 10)) * 1000
    +
    +  private val logDir = conf.get("spark.history.fs.logDirectory", null)
    +  if (logDir == null) {
    +    throw new IllegalArgumentException("Logging directory must be specified.")
    +  }
    +
    +  private val fs = Utils.getHadoopFileSystem(logDir)
    +
    +  // A timestamp of when the disk was last accessed to check for log updates
    +  private var lastLogCheckTimeMs = -1L
    +
    +  // List of applications, in order from newest to oldest.
    +  @volatile private var appList: Seq[ApplicationHistoryInfo] = Nil
    +
    +  /**
    +   * 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("LogCheckingThread") {
    +    override def run() = Utils.logUncaughtExceptions {
    +      while (true) {
    +        val now = getMonotonicTimeMs()
    +        if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) {
    +          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(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now)
    +        }
    +        checkForLogs()
    +      }
    +    }
    +  }
    +
    +  initialize()
    +
    +  private def initialize() {
    +    // Validate the log directory.
    +    val path = new Path(logDir)
    +    if (!fs.exists(path)) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified does not exist: %s".format(logDir))
    +    }
    +    if (!fs.getFileStatus(path).isDir) {
    +      throw new IllegalArgumentException(
    +        "Logging directory specified is not a directory: %s".format(logDir))
    +    }
    +
    +    checkForLogs()
    +    logCheckingThread.setDaemon(true)
    +    logCheckingThread.start()
    +  }
    +
    +  override def getListing() = appList
    +
    +  override def getAppUI(appId: String): SparkUI = {
    +    try {
    +      val appLogDir = fs.getFileStatus(new Path(logDir, appId))
    +      loadAppInfo(appLogDir, true)._2
    +    } catch {
    +      case e: FileNotFoundException => null
    +    }
    +  }
    +
    +  override def getConfig(): Map[String, String] =
    +    Map(("Event Log Location" -> logDir))
    +
    +  /**
    +   * Builds the application list based on the current contents of the log directory.
    +   * Tries to reuse as much of the data already in memory as possible, by not reading
    +   * applications that haven't been updated since last time the logs were checked.
    +   */
    +  private def checkForLogs() = {
    +    lastLogCheckTimeMs = getMonotonicTimeMs()
    +    logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs))
    +    try {
    +      val logStatus = fs.listStatus(new Path(logDir))
    +      val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
    +      val logInfos = logDirs
    +        .filter {
    +            dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))
    +          }
    +
    +      val currentApps = Map[String, ApplicationHistoryInfo](
    +        appList.map(app => (app.id -> app)):_*)
    +
    +      // For any application that either (i) is not listed or (ii) has changed since the last time
    +      // the listing was created (defined by the log dir's modification time), load the app's info.
    +      // Otherwise just reuse what's already in memory.
    +      val newApps = new mutable.ArrayBuffer[ApplicationHistoryInfo](logInfos.size)
    +      for (dir <- logInfos) {
    +        val curr = currentApps.getOrElse(dir.getPath().getName(), null)
    +        if (curr == null || curr.lastUpdated < getModificationTime(dir)) {
    +          try {
    +            newApps += loadAppInfo(dir, false)._1
    +          } catch {
    +            case e: Exception => logError(s"Failed to load app info from directory $dir.")
    +          }
    +        } else {
    +          newApps += curr
    +        }
    +      }
    +
    +      appList = newApps.sortBy { info => -info.endTime }
    +    } catch {
    +      case t: Throwable => logError("Exception in checking for event log updates", t)
    +    }
    +  }
    +
    +  /**
    +   * Parse the application's logs to find out the information we need to build the
    +   * listing page.
    +   *
    +   * When creating the listing of available apps, there is no need to load the whole UI for the
    +   * application. The UI is requested by the HistoryServer (by calling getAppInfo()) when the user
    +   * clicks on a specific application.
    +   *
    +   * @param logDir Directory with application's log files.
    +   * @param renderUI Whether to create the SparkUI for the application.
    +   * @return A 2-tuple `(app info, ui)`. `ui` will be null if `renderUI` is false.
    +   */
    +  private def loadAppInfo(logDir: FileStatus, renderUI: Boolean) = {
    +    val elogInfo = EventLoggingListener.parseLoggingInfo(logDir.getPath(), fs)
    +    val path = logDir.getPath
    +    val appId = path.getName
    +    val replayBus = new ReplayListenerBus(elogInfo.logPaths, fs, elogInfo.compressionCodec)
    +    val appListener = new ApplicationEventListener
    +    replayBus.addListener(appListener)
    +
    +    val ui: SparkUI = if (renderUI) {
    +        val conf = this.conf.clone()
    +        val appSecManager = new SecurityManager(conf)
    +        new SparkUI(conf, appSecManager, replayBus, appId, "/history/" + appId)
    +        // Do not call ui.bind() to avoid creating a new server for each application
    +      } else {
    +        null
    +      }
    +
    +    replayBus.replay()
    +    val appInfo = ApplicationHistoryInfo(appId,
    --- End diff --
    
    minor: move `appId` to next line
    ```
    ApplicationHistoryInfo(
      appId,
      ...)
    ```


---
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-1768] History server enhancements.

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

    https://github.com/apache/spark/pull/718#discussion_r13618737
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
    @@ -38,56 +39,70 @@ import org.apache.spark.util.Utils
      * 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,
    +    provider: ApplicationHistoryProvider,
         securityManager: SecurityManager,
    -    conf: SparkConf)
    -  extends WebUI(securityManager, HistoryServer.WEB_UI_PORT, conf) with Logging {
    +    port: Int)
    +  extends WebUI(securityManager, port, conf) with Logging {
     
    -  import HistoryServer._
    +  // How many applications to retain
    +  private val retainedApplications = conf.getInt("spark.history.retainedApplications", 50)
     
    -  private val fileSystem = Utils.getHadoopFileSystem(baseLogDir)
       private val localHost = Utils.localHostName()
    -  private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHost)
     
    -  // A timestamp of when the disk was last accessed to check for log updates
    -  private var lastLogCheckTime = -1L
    +  private val appLoader = new CacheLoader[String, SparkUI] {
    +    override def load(key: String): SparkUI = {
    +      val ui = provider.getAppUI(key)
    +      if (ui == null) {
    +        throw new NoSuchElementException()
    +      }
    +      attachSparkUI(ui)
    +      ui
    +    }
    +  }
     
    -  // Number of completed applications found in this directory
    -  private var numCompletedApplications = 0
    +  private val appCache = CacheBuilder.newBuilder()
    +    .maximumSize(retainedApplications)
    +    .removalListener(new RemovalListener[String, SparkUI] {
    +      override def onRemoval(rm: RemovalNotification[String, SparkUI]) = {
    +        detachSparkUI(rm.getValue())
    +      }
    +    })
    +    .build(appLoader)
    +
    +  private val loaderServlet = new HttpServlet {
    +    protected override def doGet(req: HttpServletRequest, res: HttpServletResponse): Unit = {
    +      val parts = Option(req.getPathInfo()).getOrElse("").split("/")
    +      if (parts.length < 2) {
    +        res.sendError(HttpServletResponse.SC_BAD_REQUEST,
    +          s"Unexpected path info in request (URI = ${req.getRequestURI()}")
    --- End diff --
    
    I'd rather not make user errors show up in the log. It makes it too easy to flood the log.
    
    With this, the user that caused the error will be able to see the message.


---
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.
---