1717
1818package org .apache .spark .ui .jobs
1919
20- import scala .collection .mutable .ListBuffer
20+ import scala .collection .mutable .{ HashMap , ListBuffer }
2121import scala .xml .{Node , NodeSeq , Unparsed }
2222
2323import java .util .Date
@@ -52,14 +52,16 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
5252 <text x =" 35px" y =" 47px" >Executor Removed </text >
5353 </svg ></div >.toString.filter(_ != '\n ' )
5454
55- private def getlastStageDescription (job : JobUIData ) = {
55+ private def getLastStageNameAndDescription (job : JobUIData ) : ( String , String ) = {
5656 val lastStageInfo = Option (job.stageIds)
5757 .filter(_.nonEmpty)
58- .flatMap { ids => parent.listener .stageIdToInfo.get(ids.max)}
58+ .flatMap { ids => parent.jobProgresslistener .stageIdToInfo.get(ids.max)}
5959 val lastStageData = lastStageInfo.flatMap { s =>
60- parent.listener .stageIdToData.get((s.stageId, s.attemptId))
60+ parent.jobProgresslistener .stageIdToData.get((s.stageId, s.attemptId))
6161 }
62- lastStageData.flatMap(_.description).getOrElse(" " )
62+ val name = lastStageInfo.map(_.name).getOrElse(" (Unknown Stage Name)" )
63+ val description = lastStageData.flatMap(_.description).getOrElse(" " )
64+ (name, description)
6365 }
6466
6567 private def makeJobEvent (jobUIDatas : Seq [JobUIData ]): Seq [String ] = {
@@ -68,7 +70,8 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
6870 }.map { jobUIData =>
6971 val jobId = jobUIData.jobId
7072 val status = jobUIData.status
71- val jobDescription = getlastStageDescription(jobUIData)
73+ val (jobName, jobDescription) = getLastStageNameAndDescription(jobUIData)
74+ val displayJobDescription = if (jobDescription.isEmpty) jobName else jobDescription
7275 val submissionTime = jobUIData.submissionTime.get
7376 val completionTimeOpt = jobUIData.completionTime
7477 val completionTime = completionTimeOpt.getOrElse(System .currentTimeMillis())
@@ -86,8 +89,8 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
8689 | 'start': new Date( ${submissionTime}),
8790 | 'end': new Date( ${completionTime}),
8891 | 'content': '<div class="application-timeline-content">' +
89- | ' ${jobDescription } (Job ${jobId})</div>',
90- | 'title': ' ${jobDescription } (Job ${jobId}) \\ nStatus: ${status}\\ n' +
92+ | ' ${displayJobDescription } (Job ${jobId})</div>',
93+ | 'title': ' ${displayJobDescription } (Job ${jobId}) \\ nStatus: ${status}\\ n' +
9194 | 'Submission Time: ${UIUtils .formatDate(new Date (submissionTime))}' +
9295 | ' ${
9396 if (status != JobExecutionStatus .RUNNING ) {
@@ -102,23 +105,21 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
102105 }
103106 }
104107
105- private def makeExecutorEvent (executorUIDatas : Seq [ ExecutorUIData ]): Seq [String ] = {
108+ private def makeExecutorEvent (executorUIDatas : HashMap [ String , ExecutorUIData ]): Seq [String ] = {
106109 val events = ListBuffer [String ]()
107- executorUIDatas.foreach { event =>
108-
109- if (event.startTime.isDefined) {
110- val addedEvent =
111- s """
112- |{
113- | 'className': 'executor added',
114- | 'group': 'executors',
115- | 'start': new Date( ${event.startTime.get}),
116- | 'content': '<div>Executor ${event.executorId} added</div>',
117- | 'title': 'Added at ${UIUtils .formatDate(new Date (event.startTime.get))}'
118- |}
119- """ .stripMargin
120- events += addedEvent
121- }
110+ executorUIDatas.foreach {
111+ case (executorId, event) =>
112+ val addedEvent =
113+ s """
114+ |{
115+ | 'className': 'executor added',
116+ | 'group': 'executors',
117+ | 'start': new Date( ${event.startTime}),
118+ | 'content': '<div>Executor ${executorId} added</div>',
119+ | 'title': 'Added at ${UIUtils .formatDate(new Date (event.startTime))}'
120+ |}
121+ """ .stripMargin
122+ events += addedEvent
122123
123124 if (event.finishTime.isDefined) {
124125 val removedEvent =
@@ -127,7 +128,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
127128 | 'className': 'executor removed',
128129 | 'group': 'executors',
129130 | 'start': new Date( ${event.finishTime.get}),
130- | 'content': '<div>Executor ${event. executorId} removed</div>',
131+ | 'content': '<div>Executor ${executorId} removed</div>',
131132 | 'title': 'Removed at ${UIUtils .formatDate(new Date (event.finishTime.get))}' +
132133 | ' ${
133134 if (event.finishReason.isDefined) {
@@ -146,7 +147,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
146147
147148 private def makeTimeline (
148149 jobs : Seq [JobUIData ],
149- executors : Seq [ ExecutorUIData ],
150+ executors : HashMap [ String , ExecutorUIData ],
150151 startTime : Long ): Seq [Node ] = {
151152
152153 val jobEventJsonAsStrSeq = makeJobEvent(jobs)
@@ -169,16 +170,17 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
169170 val eventArrayAsStr =
170171 (jobEventJsonAsStrSeq ++ executorEventJsonAsStrSeq).mkString(" [" , " ," , " ]" )
171172
172- <div class =" control-panel" >
173- <div id =" application-timeline-zoom-lock" >
174- <input type =" checkbox" checked =" checked" ></input >
175- <span >Zoom Lock </span >
173+ <div id =" application-timeline" >
174+ <div class =" control-panel" >
175+ <div id =" application-timeline-zoom-lock" >
176+ <input type =" checkbox" checked =" checked" ></input >
177+ <span >Zoom Lock </span >
178+ </div >
176179 </div >
177180 </div > ++
178- <div id =" application-timeline" ></div > ++
179181 <script type =" text/javascript" >
180182 {Unparsed (s " drawApplicationTimeline( ${groupJsonArrayAsStr}, " +
181- s " ${eventArrayAsStr}, ${startTime}); " )}
183+ s " ${eventArrayAsStr}, ${startTime}); " )}
182184 </script >
183185 }
184186
@@ -195,12 +197,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
195197 }
196198
197199 def makeRow (job : JobUIData ): Seq [Node ] = {
198- val lastStageInfo = Option (job.stageIds)
199- .filter(_.nonEmpty)
200- .flatMap { ids => parent.listener.stageIdToInfo.get(ids.max) }
201-
202- val lastStageName = lastStageInfo.map(_.name).getOrElse(" (Unknown Stage Name)" )
203- val lastStageDescription = getlastStageDescription(job)
200+ val (lastStageName, lastStageDescription) = getLastStageNameAndDescription(job)
204201 val duration : Option [Long ] = {
205202 job.submissionTime.map { start =>
206203 val end = job.completionTime.getOrElse(System .currentTimeMillis())
@@ -245,7 +242,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
245242 }
246243
247244 def render (request : HttpServletRequest ): Seq [Node ] = {
248- val listener = parent.listener
245+ val listener = parent.jobProgresslistener
249246 listener.synchronized {
250247 val startTime = listener.startTime
251248 val activeJobs = listener.activeJobs.values.toSeq
@@ -305,8 +302,10 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
305302 </div >
306303
307304 var content = summary
308- content ++= <h4 >Events on Application Timeline </h4 > ++
309- makeTimeline(activeJobs ++ completedJobs ++ failedJobs, listener.executors, startTime)
305+ val executorListener = parent.executorListener
306+ content ++= <h4 >Event Timeline </h4 > ++
307+ makeTimeline(activeJobs ++ completedJobs ++ failedJobs,
308+ executorListener.executorIdToData, startTime)
310309
311310 if (shouldShowActiveJobs) {
312311 content ++= <h4 id =" active" >Active Jobs ({activeJobs.size})</h4 > ++
0 commit comments