Skip to content

Commit da8bd14

Browse files
committed
Made submissionTime in SparkListenerJobStartas and completionTime in SparkListenerJobEnd as regular Long
Added a test case for checking backward compatibility
1 parent 0412a6a commit da8bd14

File tree

9 files changed

+49
-29
lines changed

9 files changed

+49
-29
lines changed

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

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -661,7 +661,7 @@ class DAGScheduler(
661661
// completion events or stage abort
662662
stageIdToStage -= s.id
663663
jobIdToStageIds -= job.jobId
664-
listenerBus.post(SparkListenerJobEnd(job.jobId, Some(clock.getTime()), jobResult))
664+
listenerBus.post(SparkListenerJobEnd(job.jobId, clock.getTime(), jobResult))
665665
}
666666
}
667667

@@ -710,7 +710,7 @@ class DAGScheduler(
710710
stage.latestInfo.stageFailed(stageFailedMessage)
711711
listenerBus.post(SparkListenerStageCompleted(stage.latestInfo))
712712
}
713-
listenerBus.post(SparkListenerJobEnd(job.jobId, Some(clock.getTime()), JobFailed(error)))
713+
listenerBus.post(SparkListenerJobEnd(job.jobId, clock.getTime(), JobFailed(error)))
714714
}
715715
}
716716

@@ -749,7 +749,7 @@ class DAGScheduler(
749749
logInfo("Missing parents: " + getMissingParentStages(finalStage))
750750
val shouldRunLocally =
751751
localExecutionEnabled && allowLocal && finalStage.parents.isEmpty && partitions.length == 1
752-
val jobSubmissionTime = Some(clock.getTime())
752+
val jobSubmissionTime = clock.getTime()
753753
if (shouldRunLocally) {
754754
// Compute very short actions like first() or take() with no parent stages locally.
755755
listenerBus.post(
@@ -969,7 +969,7 @@ class DAGScheduler(
969969
markStageAsFinished(stage)
970970
cleanupStateForJobAndIndependentStages(job)
971971
listenerBus.post(
972-
SparkListenerJobEnd(job.jobId, Some(clock.getTime()), JobSucceeded))
972+
SparkListenerJobEnd(job.jobId, clock.getTime(), JobSucceeded))
973973
}
974974

975975
// taskSucceeded runs some user code that might throw an exception. Make sure
@@ -1238,7 +1238,7 @@ class DAGScheduler(
12381238
if (ableToCancelStages) {
12391239
job.listener.jobFailed(error)
12401240
cleanupStateForJobAndIndependentStages(job)
1241-
listenerBus.post(SparkListenerJobEnd(job.jobId, Some(clock.getTime()), JobFailed(error)))
1241+
listenerBus.post(SparkListenerJobEnd(job.jobId, clock.getTime(), JobFailed(error)))
12421242
}
12431243
}
12441244

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

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -58,7 +58,7 @@ case class SparkListenerTaskEnd(
5858
@DeveloperApi
5959
case class SparkListenerJobStart(
6060
jobId: Int,
61-
time: Option[Long],
61+
time: Long,
6262
stageInfos: Seq[StageInfo],
6363
properties: Properties = null)
6464
extends SparkListenerEvent {
@@ -70,7 +70,7 @@ case class SparkListenerJobStart(
7070
@DeveloperApi
7171
case class SparkListenerJobEnd(
7272
jobId: Int,
73-
time: Option[Long],
73+
time: Long,
7474
jobResult: JobResult)
7575
extends SparkListenerEvent
7676

core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala

Lines changed: 7 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,6 @@ import scala.xml.{Node, NodeSeq}
2121

2222
import javax.servlet.http.HttpServletRequest
2323

24-
import org.apache.spark.JobExecutionStatus
2524
import org.apache.spark.ui.{WebUIPage, UIUtils}
2625
import org.apache.spark.ui.jobs.UIData.JobUIData
2726

@@ -51,13 +50,13 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
5150
val lastStageName = lastStageInfo.map(_.name).getOrElse("(Unknown Stage Name)")
5251
val lastStageDescription = lastStageData.flatMap(_.description).getOrElse("")
5352
val duration: Option[Long] = {
54-
job.startTime.map { start =>
55-
val end = job.endTime.getOrElse(System.currentTimeMillis())
53+
job.submissionTime.map { start =>
54+
val end = job.completionTime.getOrElse(System.currentTimeMillis())
5655
end - start
5756
}
5857
}
5958
val formattedDuration = duration.map(d => UIUtils.formatDuration(d)).getOrElse("Unknown")
60-
val formattedSubmissionTime = job.startTime.map(UIUtils.formatDate).getOrElse("Unknown")
59+
val formattedSubmissionTime = job.submissionTime.map(UIUtils.formatDate).getOrElse("Unknown")
6160
val detailUrl =
6261
"%s/jobs/job?id=%s".format(UIUtils.prependBaseUri(parent.basePath), job.jobId)
6362
<tr>
@@ -68,7 +67,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
6867
<div><em>{lastStageDescription}</em></div>
6968
<a href={detailUrl}>{lastStageName}</a>
7069
</td>
71-
<td sorttable_customkey={job.startTime.getOrElse(-1).toString}>
70+
<td sorttable_customkey={job.submissionTime.getOrElse(-1).toString}>
7271
{formattedSubmissionTime}
7372
</td>
7473
<td sorttable_customkey={duration.getOrElse(-1).toString}>{formattedDuration}</td>
@@ -101,11 +100,11 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
101100
val now = System.currentTimeMillis
102101

103102
val activeJobsTable =
104-
jobsTable(activeJobs.sortBy(_.startTime.getOrElse(-1L)).reverse)
103+
jobsTable(activeJobs.sortBy(_.submissionTime.getOrElse(-1L)).reverse)
105104
val completedJobsTable =
106-
jobsTable(completedJobs.sortBy(_.endTime.getOrElse(-1L)).reverse)
105+
jobsTable(completedJobs.sortBy(_.completionTime.getOrElse(-1L)).reverse)
107106
val failedJobsTable =
108-
jobsTable(failedJobs.sortBy(_.endTime.getOrElse(-1L)).reverse)
107+
jobsTable(failedJobs.sortBy(_.completionTime.getOrElse(-1L)).reverse)
109108

110109
val shouldShowActiveJobs = activeJobs.nonEmpty
111110
val shouldShowCompletedJobs = completedJobs.nonEmpty

core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -153,8 +153,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
153153
val jobData: JobUIData =
154154
new JobUIData(
155155
jobId = jobStart.jobId,
156-
startTime = jobStart.time,
157-
endTime = None,
156+
submissionTime = Option(jobStart.time).filter(_ >= 0),
158157
stageIds = jobStart.stageIds,
159158
jobGroup = jobGroup,
160159
status = JobExecutionStatus.RUNNING)
@@ -186,7 +185,8 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
186185
logWarning(s"Job completed for unknown job ${jobEnd.jobId}")
187186
new JobUIData(jobId = jobEnd.jobId)
188187
}
189-
jobData.endTime = jobEnd.time
188+
jobData.completionTime = Option(jobEnd.time).filter(_ >= 0)
189+
190190
jobEnd.jobResult match {
191191
case JobSucceeded =>
192192
completedJobs += jobData
@@ -309,7 +309,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
309309
override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized {
310310
val info = taskEnd.taskInfo
311311
// If stage attempt id is -1, it means the DAGScheduler had no idea which attempt this task
312-
// compeletion event is for. Let's just drop it here. This means we might have some speculation
312+
// completion event is for. Let's just drop it here. This means we might have some speculation
313313
// tasks on the web ui that's never marked as complete.
314314
if (info != null && taskEnd.stageAttemptId != -1) {
315315
val stageData = stageIdToData.getOrElseUpdate((taskEnd.stageId, taskEnd.stageAttemptId), {

core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -40,15 +40,15 @@ private[jobs] object UIData {
4040

4141
class JobUIData(
4242
var jobId: Int = -1,
43-
var startTime: Option[Long] = None,
44-
var endTime: Option[Long] = None,
43+
var submissionTime: Option[Long] = None,
44+
var completionTime: Option[Long] = None,
4545
var stageIds: Seq[Int] = Seq.empty,
4646
var jobGroup: Option[String] = None,
4747
var status: JobExecutionStatus = JobExecutionStatus.UNKNOWN,
4848
/* Tasks */
4949
// `numTasks` is a potential underestimate of the true number of tasks that this job will run.
5050
// This may be an underestimate because the job start event references all of the result
51-
// stages's transitive stage dependencies, but some of these stages might be skipped if their
51+
// stages' transitive stage dependencies, but some of these stages might be skipped if their
5252
// output is available from earlier runs.
5353
// See https://github.com/apache/spark/pull/3009 for a more extensive discussion.
5454
var numTasks: Int = 0,

core/src/main/scala/org/apache/spark/util/JsonProtocol.scala

Lines changed: 5 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -30,6 +30,7 @@ import org.apache.spark.executor._
3030
import org.apache.spark.scheduler._
3131
import org.apache.spark.storage._
3232
import org.apache.spark._
33+
import org.apache.hadoop.hdfs.web.JsonUtil
3334

3435
/**
3536
* Serializes SparkListener events to/from JSON. This protocol provides strong backwards-
@@ -471,7 +472,8 @@ private[spark] object JsonProtocol {
471472

472473
def jobStartFromJson(json: JValue): SparkListenerJobStart = {
473474
val jobId = (json \ "Job ID").extract[Int]
474-
val submissionTime = (json \ "Submission Time").extractOpt[Long]
475+
val submissionTime =
476+
Utils.jsonOption(json \ "Submission Time").map(_.extract[Long]).getOrElse(-1L)
475477
val stageIds = (json \ "Stage IDs").extract[List[JValue]].map(_.extract[Int])
476478
val properties = propertiesFromJson(json \ "Properties")
477479
// The "Stage Infos" field was added in Spark 1.2.0
@@ -484,7 +486,8 @@ private[spark] object JsonProtocol {
484486

485487
def jobEndFromJson(json: JValue): SparkListenerJobEnd = {
486488
val jobId = (json \ "Job ID").extract[Int]
487-
val completionTime = (json \ "Completion Time").extractOpt[Long]
489+
val completionTime =
490+
Utils.jsonOption(json \ "Completion Time").map(_.extract[Long]).getOrElse(-1L)
488491
val jobResult = jobResultFromJson(json \ "Job Result")
489492
SparkListenerJobEnd(jobId, completionTime, jobResult)
490493
}

core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -34,7 +34,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers
3434
/** Length of time to wait while draining listener events. */
3535
val WAIT_TIMEOUT_MILLIS = 10000
3636

37-
val jobCompletionTime = Option(1421191296660L)
37+
val jobCompletionTime = 1421191296660L
3838

3939
before {
4040
sc = new SparkContext("local", "SparkListenerSuite")

core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -28,8 +28,8 @@ import org.apache.spark.util.Utils
2828

2929
class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matchers {
3030

31-
val jobSubmissionTime = Option(1421191042750L)
32-
val jobCompletionTime = Option(1421191296660L)
31+
val jobSubmissionTime = 1421191042750L
32+
val jobCompletionTime = 1421191296660L
3333

3434
private def createStageStartEvent(stageId: Int) = {
3535
val stageInfo = new StageInfo(stageId, 0, stageId.toString, 0, null, "")

core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala

Lines changed: 20 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -33,8 +33,8 @@ import org.apache.spark.storage._
3333

3434
class JsonProtocolSuite extends FunSuite {
3535

36-
val jobSubmissionTime = Option(1421191042750L)
37-
val jobCompletionTime = Option(1421191296660L)
36+
val jobSubmissionTime = 1421191042750L
37+
val jobCompletionTime = 1421191296660L
3838

3939
test("SparkListenerEvent") {
4040
val stageSubmitted =
@@ -250,6 +250,24 @@ class JsonProtocolSuite extends FunSuite {
250250
assertEquals(expectedJobStart, JsonProtocol.jobStartFromJson(oldEvent))
251251
}
252252

253+
test("SparkListenerJobStart and SparkListenerJobEnd backward compatibility") {
254+
// Prior to Spark 1.3.0, SparkListenerJobStart did not have a "Submission Time" property.
255+
// Also, SparkListenerJobEnd did not have a "Completion Time" property.
256+
val stageIds = Seq[Int](1, 2, 3, 4)
257+
val stageInfos = stageIds.map(x => makeStageInfo(x * 10, x * 20, x * 30, x * 40, x * 50))
258+
val jobStart = SparkListenerJobStart(11, jobSubmissionTime, stageInfos, properties)
259+
val oldStartEvent = JsonProtocol.jobStartToJson(jobStart)
260+
.removeField({ _._1 == "Submission Time"})
261+
val expectedJobStart = SparkListenerJobStart(11, -1, stageInfos, properties)
262+
assertEquals(expectedJobStart, JsonProtocol.jobStartFromJson(oldStartEvent))
263+
264+
val jobEnd = SparkListenerJobEnd(11, jobCompletionTime, JobSucceeded)
265+
val oldEndEvent = JsonProtocol.jobEndToJson(jobEnd)
266+
.removeField({ _._1 == "Completion Time"})
267+
val expectedJobEnd = SparkListenerJobEnd(11, -1, JobSucceeded)
268+
assertEquals(expectedJobEnd, JsonProtocol.jobEndFromJson(oldEndEvent))
269+
}
270+
253271
/** -------------------------- *
254272
| Helper test running methods |
255273
* --------------------------- */

0 commit comments

Comments
 (0)