Skip to content

Commit b158d98

Browse files
committed
Address Patrick's comments
1 parent 69d1b41 commit b158d98

File tree

6 files changed

+35
-27
lines changed

6 files changed

+35
-27
lines changed

core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala

Lines changed: 14 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -32,7 +32,7 @@ import org.apache.spark.ui.JettyUtils._
3232
import org.apache.spark.util.Utils
3333

3434
/**
35-
* A web server that renders SparkUIs of finished applications.
35+
* A web server that renders SparkUIs of completed applications.
3636
*
3737
* For the standalone mode, MasterWebUI already achieves this functionality. Thus, the
3838
* main use case of the HistoryServer is in other deploy modes (e.g. Yarn or Mesos).
@@ -61,8 +61,8 @@ class HistoryServer(
6161
// A timestamp of when the disk was last accessed to check for log updates
6262
private var lastLogCheckTime = -1L
6363

64-
// Number of complete applications found in this directory
65-
private var numApplicationsTotal = 0
64+
// Number of completed applications found in this directory
65+
private var numCompletedApplications = 0
6666

6767
@volatile private var stopped = false
6868

@@ -125,11 +125,11 @@ class HistoryServer(
125125
* Check for any updates to event logs in the base directory. This is only effective once
126126
* the server has been bound.
127127
*
128-
* If a new finished application is found, the server renders the associated SparkUI
128+
* If a new completed application is found, the server renders the associated SparkUI
129129
* from the application's event logs, attaches this UI to itself, and stores metadata
130130
* information for this application.
131131
*
132-
* If the logs for an existing finished application are no longer found, the server
132+
* If the logs for an existing completed application are no longer found, the server
133133
* removes all associated information and detaches the SparkUI.
134134
*/
135135
def checkForLogs() = synchronized {
@@ -164,8 +164,8 @@ class HistoryServer(
164164
}
165165
}
166166

167-
// Track the total number of complete applications observed this round
168-
numApplicationsTotal = logInfos.size
167+
// Track the total number of completed applications observed this round
168+
numCompletedApplications = logInfos.size
169169

170170
} catch {
171171
case t: Throwable => logError("Exception in checking for event log updates", t)
@@ -176,10 +176,10 @@ class HistoryServer(
176176
}
177177

178178
/**
179-
* Render a new SparkUI from the event logs if the associated application is finished.
179+
* Render a new SparkUI from the event logs if the associated application is completed.
180180
*
181181
* HistoryServer looks for a special file that indicates application completion in the given
182-
* directory. If this file exists, the associated application is regarded to be complete, in
182+
* directory. If this file exists, the associated application is regarded to be completed, in
183183
* which case the server proceeds to render the SparkUI. Otherwise, the server does nothing.
184184
*/
185185
private def renderSparkUI(logDir: FileStatus, logInfo: EventLoggingInfo) {
@@ -200,7 +200,7 @@ class HistoryServer(
200200
val startTime = appListener.startTime
201201
val endTime = appListener.endTime
202202
val lastUpdated = getModificationTime(logDir)
203-
ui.setAppName(appName + " (finished)")
203+
ui.setAppName(appName + " (completed)")
204204
appIdToInfo(appId) = ApplicationHistoryInfo(appId, appName, startTime, endTime,
205205
lastUpdated, sparkUser, path, ui)
206206
}
@@ -216,14 +216,14 @@ class HistoryServer(
216216
/** Return the address of this server. */
217217
def getAddress: String = "http://" + publicHost + ":" + boundPort
218218

219-
/** Return the total number of application logs found, whether or not the UI is retained. */
220-
def getNumApplications: Int = numApplicationsTotal
219+
/** Return the number of completed applications found, whether or not the UI is rendered. */
220+
def getNumApplications: Int = numCompletedApplications
221221

222222
/** Return when this directory was last modified. */
223223
private def getModificationTime(dir: FileStatus): Long = {
224224
try {
225225
val logFiles = fileSystem.listStatus(dir.getPath)
226-
if (logFiles != null) {
226+
if (logFiles != null && !logFiles.isEmpty) {
227227
logFiles.map(_.getModificationTime).max
228228
} else {
229229
dir.getModificationTime
@@ -283,5 +283,5 @@ private[spark] case class ApplicationHistoryInfo(
283283
logDirPath: Path,
284284
ui: SparkUI) {
285285
def started = startTime != -1
286-
def finished = endTime != -1
286+
def completed = endTime != -1
287287
}

core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala

Lines changed: 6 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -55,8 +55,12 @@ private[spark] class HistoryServerArguments(args: Array[String]) {
5555
}
5656
val fileSystem = Utils.getHadoopFileSystem(new URI(logDir))
5757
val path = new Path(logDir)
58-
if (!fileSystem.exists(path) || !fileSystem.getFileStatus(path).isDir) {
59-
System.err.println("Logging directory specified is invalid: %s".format(logDir))
58+
if (!fileSystem.exists(path)) {
59+
System.err.println("Logging directory specified does not exist: %s".format(logDir))
60+
printUsageAndExit(1)
61+
}
62+
if (!fileSystem.getFileStatus(path).isDir) {
63+
System.err.println("Logging directory specified is not a directory: %s".format(logDir))
6064
printUsageAndExit(1)
6165
}
6266
}

core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -63,8 +63,8 @@ private[spark] class IndexPage(parent: HistoryServer) {
6363
val appName = if (info.started) info.name else info.logDirPath.getName
6464
val uiAddress = parent.getAddress + info.ui.basePath
6565
val startTime = if (info.started) WebUI.formatDate(info.startTime) else "Not started"
66-
val endTime = if (info.finished) WebUI.formatDate(info.endTime) else "Not finished"
67-
val difference = if (info.started && info.finished) info.endTime - info.startTime else -1L
66+
val endTime = if (info.completed) WebUI.formatDate(info.endTime) else "Not finished"
67+
val difference = if (info.started && info.completed) info.endTime - info.startTime else -1L
6868
val duration = if (difference > 0) WebUI.formatDuration(difference) else "---"
6969
val sparkUser = if (info.started) info.sparkUser else "Unknown user"
7070
val logDirectory = info.logDirPath.getName

core/src/main/scala/org/apache/spark/deploy/master/Master.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -667,7 +667,7 @@ private[spark] class Master(
667667
if (!eventLogPaths.isEmpty) {
668668
try {
669669
val replayBus = new ReplayListenerBus(eventLogPaths, fileSystem, compressionCodec)
670-
val ui = new SparkUI(replayBus, appName + " (finished)", "/history/" + app.id)
670+
val ui = new SparkUI(replayBus, appName + " (completed)", "/history/" + app.id)
671671
ui.start()
672672
replayBus.replay()
673673
app.desc.appUiUrl = ui.basePath

core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -20,9 +20,8 @@ package org.apache.spark.scheduler
2020
/**
2121
* A simple listener for application events.
2222
*
23-
* This listener assumes at most one of each of SparkListenerApplicationStart and
24-
* SparkListenerApplicationEnd will be received. Otherwise, only the latest event
25-
* of each type will take effect.
23+
* This listener expects to hear events from a single application only. If events
24+
* from multiple applications are seen, the behavior is unspecified.
2625
*/
2726
private[spark] class ApplicationEventListener extends SparkListener {
2827
var appName = "<Not Started>"

core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala

Lines changed: 10 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,8 @@
1717

1818
package org.apache.spark.scheduler
1919

20+
import scala.collection.mutable
21+
2022
import org.apache.hadoop.fs.{FileSystem, Path}
2123
import org.json4s.jackson.JsonMethods._
2224

@@ -118,6 +120,9 @@ private[spark] object EventLoggingListener extends Logging {
118120
val COMPRESSION_CODEC_PREFIX = "COMPRESSION_CODEC_"
119121
val APPLICATION_COMPLETE = "APPLICATION_COMPLETE"
120122

123+
// A cache for compression codecs to avoid creating the same codec many times
124+
private val codecMap = new mutable.HashMap[String, CompressionCodec]
125+
121126
def isEventLogFile(fileName: String): Boolean = {
122127
fileName.startsWith(LOG_PREFIX)
123128
}
@@ -174,11 +179,11 @@ private[spark] object EventLoggingListener extends Logging {
174179
compressionCodec = filePaths
175180
.find { path => isCompressionCodecFile(path.getName) }
176181
.map { path =>
177-
val codec = EventLoggingListener.parseCompressionCodec(path.getName)
178-
val conf = new SparkConf
179-
conf.set("spark.io.compression.codec", codec)
180-
CompressionCodec.createCodec(conf)
181-
},
182+
val codec = EventLoggingListener.parseCompressionCodec(path.getName)
183+
val conf = new SparkConf
184+
conf.set("spark.io.compression.codec", codec)
185+
codecMap.getOrElseUpdate(codec, CompressionCodec.createCodec(conf))
186+
},
182187
applicationComplete = filePaths.exists { path => isApplicationCompleteFile(path.getName) }
183188
)
184189
} catch {

0 commit comments

Comments
 (0)