Skip to content

Commit

Permalink
Fix unit tests failures
Browse files Browse the repository at this point in the history
  • Loading branch information
Sanket committed May 3, 2017
1 parent e7da27d commit f1fa890
Show file tree
Hide file tree
Showing 16 changed files with 81 additions and 22 deletions.
2 changes: 1 addition & 1 deletion core/src/main/scala/org/apache/spark/SparkContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -449,7 +449,7 @@ class SparkContext(config: SparkConf) extends Logging {

_ui =
if (conf.getBoolean("spark.ui.enabled", true)) {
Some(SparkUI.createLiveUI(this, _conf, listenerBus, SPARK_VERSION, _jobProgressListener,
Some(SparkUI.createLiveUI(this, _conf, listenerBus, _jobProgressListener,
_env.securityManager, appName, startTime = startTime))
} else {
// For tests, do not enable the UI
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -244,19 +244,21 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
applications.get(appId).flatMap { appInfo =>
appInfo.attempts.find(_.attemptId == attemptId).flatMap { attempt =>
val replayBus = new ReplayListenerBus()
val fileStatus = fs.getFileStatus(new Path(logDir, attempt.logPath))
val appListener = replay(fileStatus, isApplicationCompleted(fileStatus), replayBus)

val ui = {
val conf = this.conf.clone()
val appSecManager = new SecurityManager(conf)
SparkUI.createHistoryUI(conf, replayBus, appSecManager,
appListener.appSparkVersion.getOrElse(""), appInfo.name,
HistoryServer.getAttemptURI(appId, attempt.attemptId), attempt.startTime)
appInfo.name, HistoryServer.getAttemptURI(appId, attempt.attemptId),
attempt.startTime)
// Do not call ui.bind() to avoid creating a new server for each application
}

val fileStatus = fs.getFileStatus(new Path(logDir, attempt.logPath))

val appListener = replay(fileStatus, isApplicationCompleted(fileStatus), replayBus)

if (appListener.appId.isDefined) {
ui.setAppSparkVersion(appListener.appSparkVersion.getOrElse(""))
ui.getSecurityManager.setAcls(HISTORY_UI_ACLS_ENABLE)
// make sure to set admin acls before view acls so they are properly picked up
val adminAcls = HISTORY_UI_ADMIN_ACLS + "," + appListener.adminAcls.getOrElse("")
Expand Down Expand Up @@ -469,8 +471,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
lastUpdated,
appListener.sparkUser.getOrElse(NOT_STARTED),
appCompleted,
fileStatus.getLen(),
appListener.appSparkVersion.getOrElse("")
appListener.appSparkVersion.getOrElse(""),
fileStatus.getLen()
)
fileToAppInfo(logPath) = attemptInfo
logDebug(s"Application log ${attemptInfo.logPath} loaded successfully: $attemptInfo")
Expand Down Expand Up @@ -743,7 +745,6 @@ private[history] object FsHistoryProvider {
*
* @param logPath path to the log file, or, for a legacy log, its directory
* @param name application name
* @param appSparkVersion application spark version
* @param appId application ID
* @param attemptId optional attempt ID
* @param startTime start time (from playback)
Expand All @@ -764,8 +765,8 @@ private class FsApplicationAttemptInfo(
lastUpdated: Long,
sparkUser: String,
completed: Boolean,
val fileSize: Long,
appSparkVersion: String)
appSparkVersion: String,
val fileSize: Long)
extends ApplicationAttemptInfo(
attemptId, startTime, endTime, lastUpdated, sparkUser, completed, appSparkVersion) {

Expand Down
18 changes: 10 additions & 8 deletions core/src/main/scala/org/apache/spark/ui/SparkUI.scala
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,6 @@ private[spark] class SparkUI private (
val jobProgressListener: JobProgressListener,
val storageListener: StorageListener,
val operationGraphListener: RDDOperationGraphListener,
val appSparkVersion: String,
var appName: String,
val basePath: String,
val startTime: Long)
Expand All @@ -61,6 +60,8 @@ private[spark] class SparkUI private (

var appId: String = _

var appSparkVersion = ""

private var streamingJobProgressListener: Option[SparkListener] = None

/** Initialize all components of the server. */
Expand Down Expand Up @@ -94,6 +95,10 @@ private[spark] class SparkUI private (
appId = id
}

def setAppSparkVersion(version: String): Unit = {
appSparkVersion = version
}

/** Stop the server behind this web interface. Only valid after bind(). */
override def stop() {
super.stop()
Expand Down Expand Up @@ -159,25 +164,23 @@ private[spark] object SparkUI {
sc: SparkContext,
conf: SparkConf,
listenerBus: SparkListenerBus,
appSparkVersion: String,
jobProgressListener: JobProgressListener,
securityManager: SecurityManager,
appName: String,
startTime: Long): SparkUI = {
create(Some(sc), conf, listenerBus, securityManager, appSparkVersion, appName,
create(Some(sc), conf, listenerBus, securityManager, appName,
jobProgressListener = Some(jobProgressListener), startTime = startTime)
}

def createHistoryUI(
conf: SparkConf,
listenerBus: SparkListenerBus,
securityManager: SecurityManager,
appSparkVersion: String,
appName: String,
basePath: String,
startTime: Long): SparkUI = {
val sparkUI = create(None, conf, listenerBus, securityManager,
appSparkVersion, appName, basePath, startTime = startTime)
val sparkUI = create(None, conf, listenerBus, securityManager, appName, basePath,
startTime = startTime)

val listenerFactories = ServiceLoader.load(classOf[SparkHistoryListenerFactory],
Utils.getContextOrSparkClassLoader).asScala
Expand All @@ -200,7 +203,6 @@ private[spark] object SparkUI {
conf: SparkConf,
listenerBus: SparkListenerBus,
securityManager: SecurityManager,
appSparkVersion: String,
appName: String,
basePath: String = "",
jobProgressListener: Option[JobProgressListener] = None,
Expand All @@ -226,6 +228,6 @@ private[spark] object SparkUI {

new SparkUI(sc, conf, securityManager, environmentListener,
storageStatusListener, executorsListener, _jobProgressListener, storageListener,
operationGraphListener, appSparkVersion, appName, basePath, startTime)
operationGraphListener, appName, basePath, startTime)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@
"duration" : 10671,
"sparkUser" : "jose",
"completed" : true,
"appSparkVersion" : "",
"endTimeEpoch" : 1479335620587,
"startTimeEpoch" : 1479335609916,
"lastUpdatedEpoch" : 0
Expand All @@ -22,6 +23,7 @@
"duration" : 101795,
"sparkUser" : "jose",
"completed" : true,
"appSparkVersion" : "",
"endTimeEpoch" : 1479252138874,
"startTimeEpoch" : 1479252037079,
"lastUpdatedEpoch" : 0
Expand All @@ -36,6 +38,7 @@
"duration" : 10505,
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "",
"endTimeEpoch" : 1430917391398,
"startTimeEpoch" : 1430917380893,
"lastUpdatedEpoch" : 0
Expand All @@ -51,6 +54,7 @@
"duration" : 57,
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "",
"endTimeEpoch" : 1430917380950,
"startTimeEpoch" : 1430917380893,
"lastUpdatedEpoch" : 0
Expand All @@ -62,6 +66,7 @@
"duration" : 10,
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "",
"endTimeEpoch" : 1430917380890,
"startTimeEpoch" : 1430917380880,
"lastUpdatedEpoch" : 0
Expand All @@ -77,6 +82,7 @@
"duration" : 34935,
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "",
"endTimeEpoch" : 1426633945177,
"startTimeEpoch" : 1426633910242,
"lastUpdatedEpoch" : 0
Expand All @@ -88,6 +94,7 @@
"duration" : 34935,
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "",
"endTimeEpoch" : 1426533945177,
"startTimeEpoch" : 1426533910242,
"lastUpdatedEpoch" : 0
Expand All @@ -102,6 +109,7 @@
"duration" : 8635,
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "",
"endTimeEpoch" : 1425081766912,
"startTimeEpoch" : 1425081758277,
"lastUpdatedEpoch" : 0
Expand All @@ -116,6 +124,7 @@
"duration" : 9011,
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "",
"endTimeEpoch" : 1422981788731,
"startTimeEpoch" : 1422981779720,
"lastUpdatedEpoch" : 0
Expand All @@ -130,6 +139,7 @@
"duration" : 8635,
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "",
"endTimeEpoch" : 1422981766912,
"startTimeEpoch" : 1422981758277,
"lastUpdatedEpoch" : 0
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@
"duration" : 10671,
"sparkUser" : "jose",
"completed" : true,
"appSparkVersion" : "",
"endTimeEpoch" : 1479335620587,
"startTimeEpoch" : 1479335609916,
"lastUpdatedEpoch" : 0
Expand All @@ -22,6 +23,7 @@
"duration" : 101795,
"sparkUser" : "jose",
"completed" : true,
"appSparkVersion" : "",
"endTimeEpoch" : 1479252138874,
"startTimeEpoch" : 1479252037079,
"lastUpdatedEpoch" : 0
Expand All @@ -36,6 +38,7 @@
"duration" : 10505,
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "",
"endTimeEpoch" : 1430917391398,
"startTimeEpoch" : 1430917380893,
"lastUpdatedEpoch" : 0
Expand All @@ -51,6 +54,7 @@
"duration" : 57,
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "",
"endTimeEpoch" : 1430917380950,
"startTimeEpoch" : 1430917380893,
"lastUpdatedEpoch" : 0
Expand All @@ -62,6 +66,7 @@
"duration" : 10,
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "",
"endTimeEpoch" : 1430917380890,
"startTimeEpoch" : 1430917380880,
"lastUpdatedEpoch" : 0
Expand All @@ -77,6 +82,7 @@
"duration" : 34935,
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "",
"endTimeEpoch" : 1426633945177,
"startTimeEpoch" : 1426633910242,
"lastUpdatedEpoch" : 0
Expand All @@ -88,6 +94,7 @@
"duration" : 34935,
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "",
"endTimeEpoch" : 1426533945177,
"startTimeEpoch" : 1426533910242,
"lastUpdatedEpoch" : 0
Expand All @@ -102,6 +109,8 @@
"duration" : 8635,
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "",
"appSparkVersion" : "",
"endTimeEpoch" : 1425081766912,
"startTimeEpoch" : 1425081758277,
"lastUpdatedEpoch" : 0
Expand All @@ -116,6 +125,7 @@
"duration" : 9011,
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "",
"endTimeEpoch" : 1422981788731,
"startTimeEpoch" : 1422981779720,
"lastUpdatedEpoch" : 0
Expand All @@ -130,6 +140,7 @@
"duration" : 8635,
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "",
"endTimeEpoch" : 1422981766912,
"startTimeEpoch" : 1422981758277,
"lastUpdatedEpoch" : 0
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@
"duration" : 10671,
"sparkUser" : "jose",
"completed" : true,
"appSparkVersion" : "",
"endTimeEpoch" : 1479335620587,
"startTimeEpoch" : 1479335609916,
"lastUpdatedEpoch" : 0
Expand All @@ -22,6 +23,7 @@
"duration" : 101795,
"sparkUser" : "jose",
"completed" : true,
"appSparkVersion" : "",
"endTimeEpoch" : 1479252138874,
"startTimeEpoch" : 1479252037079,
"lastUpdatedEpoch" : 0
Expand All @@ -36,6 +38,7 @@
"duration" : 10505,
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "",
"endTimeEpoch" : 1430917391398,
"startTimeEpoch" : 1430917380893,
"lastUpdatedEpoch" : 0
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@
"duration" : 8635,
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "",
"endTimeEpoch" : 1422981766912,
"startTimeEpoch" : 1422981758277,
"lastUpdatedEpoch" : 0
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@
"duration" : 9011,
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "",
"endTimeEpoch" : 1422981788731,
"startTimeEpoch" : 1422981779720,
"lastUpdatedEpoch" : 0
Expand All @@ -22,6 +23,7 @@
"duration" : 8635,
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "",
"endTimeEpoch" : 1422981766912,
"startTimeEpoch" : 1422981758277,
"lastUpdatedEpoch" : 0
Expand Down
Loading

0 comments on commit f1fa890

Please sign in to comment.