Skip to content

Commit c037d25

Browse files
ajbozarthTom Graves
authored andcommitted
[SPARK-12149][WEB UI] Executor UI improvement suggestions - Color UI
Added color coding to the Executors page for Active Tasks, Failed Tasks, Completed Tasks and Task Time. Active Tasks is shaded blue with it's range based on percentage of total cores used. Failed Tasks is shaded red ranging over the first 10% of total tasks failed Completed Tasks is shaded green ranging over 10% of total tasks including failed and active tasks, but only when there are active or failed tasks on that executor. Task Time is shaded red when GC Time goes over 10% of total time with it's range directly corresponding to the percent of total time. Author: Alex Bozarth <[email protected]> Closes #10154 from ajbozarth/spark12149.
1 parent ef8fb36 commit c037d25

File tree

7 files changed

+103
-20
lines changed

7 files changed

+103
-20
lines changed

core/src/main/scala/org/apache/spark/status/api/v1/api.scala

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -55,11 +55,13 @@ class ExecutorSummary private[spark](
5555
val rddBlocks: Int,
5656
val memoryUsed: Long,
5757
val diskUsed: Long,
58+
val maxTasks: Int,
5859
val activeTasks: Int,
5960
val failedTasks: Int,
6061
val completedTasks: Int,
6162
val totalTasks: Int,
6263
val totalDuration: Long,
64+
val totalGCTime: Long,
6365
val totalInputBytes: Long,
6466
val totalShuffleRead: Long,
6567
val totalShuffleWrite: Long,

core/src/main/scala/org/apache/spark/ui/SparkUI.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -195,7 +195,7 @@ private[spark] object SparkUI {
195195

196196
val environmentListener = new EnvironmentListener
197197
val storageStatusListener = new StorageStatusListener
198-
val executorsListener = new ExecutorsListener(storageStatusListener)
198+
val executorsListener = new ExecutorsListener(storageStatusListener, conf)
199199
val storageListener = new StorageListener(storageStatusListener)
200200
val operationGraphListener = new RDDOperationGraphListener(conf)
201201

core/src/main/scala/org/apache/spark/ui/ToolTips.scala

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -87,4 +87,7 @@ private[spark] object ToolTips {
8787
multiple operations (e.g. two map() functions) if they can be pipelined. Some operations
8888
also create multiple RDDs internally. Cached RDDs are shown in green.
8989
"""
90+
91+
val TASK_TIME =
92+
"Shaded red when garbage collection (GC) time is over 10% of task time"
9093
}

core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala

Lines changed: 81 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -50,6 +50,8 @@ private[ui] class ExecutorsPage(
5050
threadDumpEnabled: Boolean)
5151
extends WebUIPage("") {
5252
private val listener = parent.listener
53+
// When GCTimePercent is edited change ToolTips.TASK_TIME to match
54+
private val GCTimePercent = 0.1
5355

5456
def render(request: HttpServletRequest): Seq[Node] = {
5557
val (storageStatusList, execInfo) = listener.synchronized {
@@ -77,7 +79,7 @@ private[ui] class ExecutorsPage(
7779
<th>Failed Tasks</th>
7880
<th>Complete Tasks</th>
7981
<th>Total Tasks</th>
80-
<th>Task Time</th>
82+
<th data-toggle="tooltip" title={ToolTips.TASK_TIME}>Task Time (GC Time)</th>
8183
<th><span data-toggle="tooltip" title={ToolTips.INPUT}>Input</span></th>
8284
<th><span data-toggle="tooltip" title={ToolTips.SHUFFLE_READ}>Shuffle Read</span></th>
8385
<th>
@@ -129,13 +131,8 @@ private[ui] class ExecutorsPage(
129131
<td sorttable_customkey={diskUsed.toString}>
130132
{Utils.bytesToString(diskUsed)}
131133
</td>
132-
<td>{info.activeTasks}</td>
133-
<td>{info.failedTasks}</td>
134-
<td>{info.completedTasks}</td>
135-
<td>{info.totalTasks}</td>
136-
<td sorttable_customkey={info.totalDuration.toString}>
137-
{Utils.msDurationToString(info.totalDuration)}
138-
</td>
134+
{taskData(info.maxTasks, info.activeTasks, info.failedTasks, info.completedTasks,
135+
info.totalTasks, info.totalDuration, info.totalGCTime)}
139136
<td sorttable_customkey={info.totalInputBytes.toString}>
140137
{Utils.bytesToString(info.totalInputBytes)}
141138
</td>
@@ -177,7 +174,6 @@ private[ui] class ExecutorsPage(
177174
val maximumMemory = execInfo.map(_.maxMemory).sum
178175
val memoryUsed = execInfo.map(_.memoryUsed).sum
179176
val diskUsed = execInfo.map(_.diskUsed).sum
180-
val totalDuration = execInfo.map(_.totalDuration).sum
181177
val totalInputBytes = execInfo.map(_.totalInputBytes).sum
182178
val totalShuffleRead = execInfo.map(_.totalShuffleRead).sum
183179
val totalShuffleWrite = execInfo.map(_.totalShuffleWrite).sum
@@ -192,13 +188,13 @@ private[ui] class ExecutorsPage(
192188
<td sorttable_customkey={diskUsed.toString}>
193189
{Utils.bytesToString(diskUsed)}
194190
</td>
195-
<td>{execInfo.map(_.activeTasks).sum}</td>
196-
<td>{execInfo.map(_.failedTasks).sum}</td>
197-
<td>{execInfo.map(_.completedTasks).sum}</td>
198-
<td>{execInfo.map(_.totalTasks).sum}</td>
199-
<td sorttable_customkey={totalDuration.toString}>
200-
{Utils.msDurationToString(totalDuration)}
201-
</td>
191+
{taskData(execInfo.map(_.maxTasks).sum,
192+
execInfo.map(_.activeTasks).sum,
193+
execInfo.map(_.failedTasks).sum,
194+
execInfo.map(_.completedTasks).sum,
195+
execInfo.map(_.totalTasks).sum,
196+
execInfo.map(_.totalDuration).sum,
197+
execInfo.map(_.totalGCTime).sum)}
202198
<td sorttable_customkey={totalInputBytes.toString}>
203199
{Utils.bytesToString(totalInputBytes)}
204200
</td>
@@ -219,7 +215,7 @@ private[ui] class ExecutorsPage(
219215
<th>Failed Tasks</th>
220216
<th>Complete Tasks</th>
221217
<th>Total Tasks</th>
222-
<th>Task Time</th>
218+
<th data-toggle="tooltip" title={ToolTips.TASK_TIME}>Task Time (GC Time)</th>
223219
<th><span data-toggle="tooltip" title={ToolTips.INPUT}>Input</span></th>
224220
<th><span data-toggle="tooltip" title={ToolTips.SHUFFLE_READ}>Shuffle Read</span></th>
225221
<th>
@@ -233,6 +229,70 @@ private[ui] class ExecutorsPage(
233229
</tbody>
234230
</table>
235231
}
232+
233+
private def taskData(
234+
maxTasks: Int,
235+
activeTasks: Int,
236+
failedTasks: Int,
237+
completedTasks: Int,
238+
totalTasks: Int,
239+
totalDuration: Long,
240+
totalGCTime: Long):
241+
Seq[Node] = {
242+
// Determine Color Opacity from 0.5-1
243+
// activeTasks range from 0 to maxTasks
244+
val activeTasksAlpha =
245+
if (maxTasks > 0) {
246+
(activeTasks.toDouble / maxTasks) * 0.5 + 0.5
247+
} else {
248+
1
249+
}
250+
// failedTasks range max at 10% failure, alpha max = 1
251+
val failedTasksAlpha =
252+
if (totalTasks > 0) {
253+
math.min(10 * failedTasks.toDouble / totalTasks, 1) * 0.5 + 0.5
254+
} else {
255+
1
256+
}
257+
// totalDuration range from 0 to 50% GC time, alpha max = 1
258+
val totalDurationAlpha =
259+
if (totalDuration > 0) {
260+
math.min(totalGCTime.toDouble / totalDuration + 0.5, 1)
261+
} else {
262+
1
263+
}
264+
265+
val tableData =
266+
<td style={
267+
if (activeTasks > 0) {
268+
"background:hsla(240, 100%, 50%, " + activeTasksAlpha + ");color:white"
269+
} else {
270+
""
271+
}
272+
}>{activeTasks}</td>
273+
<td style={
274+
if (failedTasks > 0) {
275+
"background:hsla(0, 100%, 50%, " + failedTasksAlpha + ");color:white"
276+
} else {
277+
""
278+
}
279+
}>{failedTasks}</td>
280+
<td>{completedTasks}</td>
281+
<td>{totalTasks}</td>
282+
<td sorttable_customkey={totalDuration.toString} style={
283+
// Red if GC time over GCTimePercent of total time
284+
if (totalGCTime > GCTimePercent * totalDuration) {
285+
"background:hsla(0, 100%, 50%, " + totalDurationAlpha + ");color:white"
286+
} else {
287+
""
288+
}
289+
}>
290+
{Utils.msDurationToString(totalDuration)}
291+
({Utils.msDurationToString(totalGCTime)})
292+
</td>;
293+
294+
tableData
295+
}
236296
}
237297

238298
private[spark] object ExecutorsPage {
@@ -245,11 +305,13 @@ private[spark] object ExecutorsPage {
245305
val memUsed = status.memUsed
246306
val maxMem = status.maxMem
247307
val diskUsed = status.diskUsed
308+
val maxTasks = listener.executorToTasksMax.getOrElse(execId, 0)
248309
val activeTasks = listener.executorToTasksActive.getOrElse(execId, 0)
249310
val failedTasks = listener.executorToTasksFailed.getOrElse(execId, 0)
250311
val completedTasks = listener.executorToTasksComplete.getOrElse(execId, 0)
251312
val totalTasks = activeTasks + failedTasks + completedTasks
252313
val totalDuration = listener.executorToDuration.getOrElse(execId, 0L)
314+
val totalGCTime = listener.executorToJvmGCTime.getOrElse(execId, 0L)
253315
val totalInputBytes = listener.executorToInputBytes.getOrElse(execId, 0L)
254316
val totalShuffleRead = listener.executorToShuffleRead.getOrElse(execId, 0L)
255317
val totalShuffleWrite = listener.executorToShuffleWrite.getOrElse(execId, 0L)
@@ -261,11 +323,13 @@ private[spark] object ExecutorsPage {
261323
rddBlocks,
262324
memUsed,
263325
diskUsed,
326+
maxTasks,
264327
activeTasks,
265328
failedTasks,
266329
completedTasks,
267330
totalTasks,
268331
totalDuration,
332+
totalGCTime,
269333
totalInputBytes,
270334
totalShuffleRead,
271335
totalShuffleWrite,

core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala

Lines changed: 8 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,7 @@ package org.apache.spark.ui.exec
1919

2020
import scala.collection.mutable.HashMap
2121

22-
import org.apache.spark.{ExceptionFailure, Resubmitted, SparkContext}
22+
import org.apache.spark.{ExceptionFailure, Resubmitted, SparkConf, SparkContext}
2323
import org.apache.spark.annotation.DeveloperApi
2424
import org.apache.spark.scheduler._
2525
import org.apache.spark.storage.{StorageStatus, StorageStatusListener}
@@ -43,11 +43,14 @@ private[ui] class ExecutorsTab(parent: SparkUI) extends SparkUITab(parent, "exec
4343
* A SparkListener that prepares information to be displayed on the ExecutorsTab
4444
*/
4545
@DeveloperApi
46-
class ExecutorsListener(storageStatusListener: StorageStatusListener) extends SparkListener {
46+
class ExecutorsListener(storageStatusListener: StorageStatusListener, conf: SparkConf)
47+
extends SparkListener {
48+
val executorToTasksMax = HashMap[String, Int]()
4749
val executorToTasksActive = HashMap[String, Int]()
4850
val executorToTasksComplete = HashMap[String, Int]()
4951
val executorToTasksFailed = HashMap[String, Int]()
5052
val executorToDuration = HashMap[String, Long]()
53+
val executorToJvmGCTime = HashMap[String, Long]()
5154
val executorToInputBytes = HashMap[String, Long]()
5255
val executorToInputRecords = HashMap[String, Long]()
5356
val executorToOutputBytes = HashMap[String, Long]()
@@ -62,6 +65,8 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener) extends Sp
6265
override def onExecutorAdded(executorAdded: SparkListenerExecutorAdded): Unit = synchronized {
6366
val eid = executorAdded.executorId
6467
executorToLogUrls(eid) = executorAdded.executorInfo.logUrlMap
68+
executorToTasksMax(eid) =
69+
executorAdded.executorInfo.totalCores / conf.getInt("spark.task.cpus", 1)
6570
executorIdToData(eid) = ExecutorUIData(executorAdded.time)
6671
}
6772

@@ -131,6 +136,7 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener) extends Sp
131136
executorToShuffleWrite(eid) =
132137
executorToShuffleWrite.getOrElse(eid, 0L) + shuffleWrite.bytesWritten
133138
}
139+
executorToJvmGCTime(eid) = executorToJvmGCTime.getOrElse(eid, 0L) + metrics.jvmGCTime
134140
}
135141
}
136142
}

core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -4,11 +4,13 @@
44
"rddBlocks" : 8,
55
"memoryUsed" : 28000128,
66
"diskUsed" : 0,
7+
"maxTasks" : 0,
78
"activeTasks" : 0,
89
"failedTasks" : 1,
910
"completedTasks" : 31,
1011
"totalTasks" : 32,
1112
"totalDuration" : 8820,
13+
"totalGCTime" : 352,
1214
"totalInputBytes" : 28000288,
1315
"totalShuffleRead" : 0,
1416
"totalShuffleWrite" : 13180,

project/MimaExcludes.scala

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -127,6 +127,9 @@ object MimaExcludes {
127127
ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.storage.ExternalBlockStore$"),
128128
ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.storage.ExternalBlockManager"),
129129
ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.storage.ExternalBlockStore")
130+
) ++ Seq(
131+
// SPARK-12149 Added new fields to ExecutorSummary
132+
ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.status.api.v1.ExecutorSummary.this")
130133
) ++
131134
// SPARK-12665 Remove deprecated and unused classes
132135
Seq(
@@ -301,6 +304,9 @@ object MimaExcludes {
301304
// SPARK-3580 Add getNumPartitions method to JavaRDD
302305
ProblemFilters.exclude[MissingMethodProblem](
303306
"org.apache.spark.api.java.JavaRDDLike.getNumPartitions")
307+
) ++ Seq(
308+
// SPARK-12149 Added new fields to ExecutorSummary
309+
ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.status.api.v1.ExecutorSummary.this")
304310
) ++
305311
// SPARK-11314: YARN backend moved to yarn sub-module and MiMA complains even though it's a
306312
// private class.

0 commit comments

Comments
 (0)