Skip to content

Commit

Permalink
Separate history server from history backend.
Browse files Browse the repository at this point in the history
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.
  • Loading branch information
Marcelo Vanzin committed May 28, 2014
1 parent 82eadc3 commit f96aa78
Show file tree
Hide file tree
Showing 5 changed files with 379 additions and 256 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,205 @@
/*
* 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

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 lastLogCheckTime = -1L

This comment has been minimized.

Copy link
@cmccabe

cmccabe Jun 2, 2014

It would be nice to call this something like lastLogCheckTimeMs, to make it clear that it is in milliseconds (and not seconds or something else)


// 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 (!stopped) {
val now = System.currentTimeMillis

This comment has been minimized.

Copy link
@cmccabe

cmccabe Jun 2, 2014

We should use monotonic time here, to avoid weird stuff happening if the user changes the wall-clock time. You can use System.nanoTime and divide by 10*6 (Hadoop has a utility method to get monotonic milliseconds, not sure if Spark does... a quick grep didn't find it.)

if (now - lastLogCheckTime > 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(lastLogCheckTime + UPDATE_INTERVAL_MS - now)
}
checkForLogs()
}
}
}

@volatile private var stopped = false

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.start()
}

override def stop() = {
stopped = true
logCheckingThread.interrupt()
logCheckingThread.join()
}

override def getListing(offset: Int, limit: Int): Seq[ApplicationHistoryInfo] = {
appList.get()
}

override def getAppInfo(appId: String): ApplicationHistoryInfo = {
try {
val appLogDir = fs.getFileStatus(new Path(logDir, appId))
loadAppInfo(appLogDir, true)
} catch {
case e: FileNotFoundException => null
}
}

/**
* Check for any updates to event logs in the base directory. This is only effective once
* the server has been bound.
*
* If a new completed application is found, the server renders the associated SparkUI
* from the application's event logs, attaches this UI to itself, and stores metadata
* information for this application.
*
* If the logs for an existing completed application are no longer found, the server
* removes all associated information and detaches the SparkUI.
*/
def checkForLogs() = synchronized {
lastLogCheckTime = System.currentTimeMillis
logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTime))
try {
val logStatus = fs.listStatus(new Path(logDir))
val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]()
val logInfos = logDirs
.sortBy { dir => getModificationTime(dir) }
.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.
appList.set(logInfos
.map { dir =>
val curr = currentApps.getOrElse(dir.getPath().getName(), null)
if (curr == null || curr.lastUpdated < getModificationTime(dir)) {
loadAppInfo(dir, false)
} else {
curr
}
}
.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 = {

This comment has been minimized.

Copy link
@cmccabe

cmccabe Jun 2, 2014

can we call this something like getLastChangeTimeMs or something? I find it confusing to refer to it as "directory modification time" since it seems like that isn't usually what it is (it's usually computed by looking at the children of the directory, not the directory itself.)

This comment has been minimized.

Copy link
@vanzin

vanzin Jun 2, 2014

Owner

I'll punt on this since I have a separate patch (not yet in a pr) that gets rid of this concept of last modification time or the history log directory entirely.

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

}
Original file line number Diff line number Diff line change
Expand Up @@ -26,19 +26,16 @@ import org.apache.spark.ui.{WebUIPage, UIUtils}
private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") {

def render(request: HttpServletRequest): Seq[Node] = {
val appRows = parent.appIdToInfo.values.toSeq.sortBy { app => -app.lastUpdated }
val appTable = UIUtils.listingTable(appHeader, appRow, appRows)
val apps = parent.getApplicationList(0, -1)
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>
</ul>
{
if (parent.appIdToInfo.size > 0) {
if (apps.size > 0) {
<h4>
Showing {parent.appIdToInfo.size}/{parent.getNumApplications}
Completed Application{if (parent.getNumApplications > 1) "s" else ""}
Showing {apps.size}/{apps.size}
Completed Application{if (apps.size > 1) "s" else ""}
</h4> ++
appTable
} else {
Expand All @@ -56,26 +53,23 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") {
"Completed",
"Duration",
"Spark User",
"Log Directory",
"Last Updated")

private def appRow(info: ApplicationHistoryInfo): Seq[Node] = {
val appName = if (info.started) info.name else info.logDirPath.getName
val uiAddress = parent.getAddress + info.ui.basePath
val appName = if (info.started) info.name else info.id
val uiAddress = "/history/" + info.id
val startTime = if (info.started) UIUtils.formatDate(info.startTime) else "Not started"
val endTime = if (info.completed) UIUtils.formatDate(info.endTime) else "Not completed"
val difference = if (info.started && info.completed) info.endTime - info.startTime else -1L
val duration = if (difference > 0) UIUtils.formatDuration(difference) else "---"
val sparkUser = if (info.started) info.sparkUser else "Unknown user"
val logDirectory = info.logDirPath.getName
val lastUpdated = UIUtils.formatDate(info.lastUpdated)
<tr>
<td><a href={uiAddress}>{appName}</a></td>
<td>{startTime}</td>
<td>{endTime}</td>
<td>{duration}</td>
<td>{sparkUser}</td>
<td>{logDirectory}</td>
<td>{lastUpdated}</td>
</tr>
}
Expand Down
Loading

1 comment on commit f96aa78

@cmccabe
Copy link

@cmccabe cmccabe commented on f96aa78 Jun 2, 2014

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks good.

Just a note related to your introduction. Hadoop has a variant of Filesystem#listStatus that doesn't materialize all the FileStatus objects in memory at once. It will do multiple RPCs as well, if necessary. This would be a good variant to use if we expect that the directory listing to get large over time (it sounds like we do?)

Please sign in to comment.