Skip to content

Commit a6bf3db

Browse files
zsxwingcloud-fan
authored andcommitted
[SPARK-23307][WEBUI] Sort jobs/stages/tasks/queries with the completed timestamp before cleaning up them
## What changes were proposed in this pull request? Sort jobs/stages/tasks/queries with the completed timestamp before cleaning up them to make the behavior consistent with 2.2. ## How was this patch tested? - Jenkins. - Manually ran the following codes and checked the UI for jobs/stages/tasks/queries. ``` spark.ui.retainedJobs 10 spark.ui.retainedStages 10 spark.sql.ui.retainedExecutions 10 spark.ui.retainedTasks 10 ``` ``` new Thread() { override def run() { spark.range(1, 2).foreach { i => Thread.sleep(10000) } } }.start() Thread.sleep(5000) for (_ <- 1 to 20) { new Thread() { override def run() { spark.range(1, 2).foreach { i => } } }.start() } Thread.sleep(15000) spark.range(1, 2).foreach { i => } sc.makeRDD(1 to 100, 100).foreach { i => } ``` Author: Shixiong Zhu <[email protected]> Closes #20481 from zsxwing/SPARK-23307.
1 parent 6fb3fd1 commit a6bf3db

File tree

6 files changed

+158
-10
lines changed

6 files changed

+158
-10
lines changed

core/src/main/scala/org/apache/spark/status/AppStatusListener.scala

Lines changed: 7 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -875,8 +875,8 @@ private[spark] class AppStatusListener(
875875
return
876876
}
877877

878-
val toDelete = KVUtils.viewToSeq(kvstore.view(classOf[JobDataWrapper]),
879-
countToDelete.toInt) { j =>
878+
val view = kvstore.view(classOf[JobDataWrapper]).index("completionTime").first(0L)
879+
val toDelete = KVUtils.viewToSeq(view, countToDelete.toInt) { j =>
880880
j.info.status != JobExecutionStatus.RUNNING && j.info.status != JobExecutionStatus.UNKNOWN
881881
}
882882
toDelete.foreach { j => kvstore.delete(j.getClass(), j.info.jobId) }
@@ -888,8 +888,8 @@ private[spark] class AppStatusListener(
888888
return
889889
}
890890

891-
val stages = KVUtils.viewToSeq(kvstore.view(classOf[StageDataWrapper]),
892-
countToDelete.toInt) { s =>
891+
val view = kvstore.view(classOf[StageDataWrapper]).index("completionTime").first(0L)
892+
val stages = KVUtils.viewToSeq(view, countToDelete.toInt) { s =>
893893
s.info.status != v1.StageStatus.ACTIVE && s.info.status != v1.StageStatus.PENDING
894894
}
895895

@@ -945,8 +945,9 @@ private[spark] class AppStatusListener(
945945
val countToDelete = calculateNumberToRemove(stage.savedTasks.get(), maxTasksPerStage).toInt
946946
if (countToDelete > 0) {
947947
val stageKey = Array(stage.info.stageId, stage.info.attemptNumber)
948-
val view = kvstore.view(classOf[TaskDataWrapper]).index("stage").first(stageKey)
949-
.last(stageKey)
948+
val view = kvstore.view(classOf[TaskDataWrapper])
949+
.index(TaskIndexNames.COMPLETION_TIME)
950+
.parent(stageKey)
950951

951952
// Try to delete finished tasks only.
952953
val toDelete = KVUtils.viewToSeq(view, countToDelete) { t =>

core/src/main/scala/org/apache/spark/status/storeTypes.scala

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -73,6 +73,8 @@ private[spark] class JobDataWrapper(
7373
@JsonIgnore @KVIndex
7474
private def id: Int = info.jobId
7575

76+
@JsonIgnore @KVIndex("completionTime")
77+
private def completionTime: Long = info.completionTime.map(_.getTime).getOrElse(-1L)
7678
}
7779

7880
private[spark] class StageDataWrapper(
@@ -90,6 +92,8 @@ private[spark] class StageDataWrapper(
9092
@JsonIgnore @KVIndex("active")
9193
private def active: Boolean = info.status == StageStatus.ACTIVE
9294

95+
@JsonIgnore @KVIndex("completionTime")
96+
private def completionTime: Long = info.completionTime.map(_.getTime).getOrElse(-1L)
9397
}
9498

9599
/**
@@ -134,6 +138,7 @@ private[spark] object TaskIndexNames {
134138
final val STAGE = "stage"
135139
final val STATUS = "sta"
136140
final val TASK_INDEX = "idx"
141+
final val COMPLETION_TIME = "ct"
137142
}
138143

139144
/**
@@ -337,6 +342,8 @@ private[spark] class TaskDataWrapper(
337342
@JsonIgnore @KVIndex(value = TaskIndexNames.ERROR, parent = TaskIndexNames.STAGE)
338343
private def error: String = if (errorMessage.isDefined) errorMessage.get else ""
339344

345+
@JsonIgnore @KVIndex(value = TaskIndexNames.COMPLETION_TIME, parent = TaskIndexNames.STAGE)
346+
private def completionTime: Long = launchTime + duration
340347
}
341348

342349
private[spark] class RDDStorageInfoWrapper(val info: RDDStorageInfo) {

core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala

Lines changed: 90 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1010,6 +1010,96 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
10101010
}
10111011
}
10121012

1013+
test("eviction should respect job completion time") {
1014+
val testConf = conf.clone().set(MAX_RETAINED_JOBS, 2)
1015+
val listener = new AppStatusListener(store, testConf, true)
1016+
1017+
// Start job 1 and job 2
1018+
time += 1
1019+
listener.onJobStart(SparkListenerJobStart(1, time, Nil, null))
1020+
time += 1
1021+
listener.onJobStart(SparkListenerJobStart(2, time, Nil, null))
1022+
1023+
// Stop job 2 before job 1
1024+
time += 1
1025+
listener.onJobEnd(SparkListenerJobEnd(2, time, JobSucceeded))
1026+
time += 1
1027+
listener.onJobEnd(SparkListenerJobEnd(1, time, JobSucceeded))
1028+
1029+
// Start job 3 and job 2 should be evicted.
1030+
time += 1
1031+
listener.onJobStart(SparkListenerJobStart(3, time, Nil, null))
1032+
assert(store.count(classOf[JobDataWrapper]) === 2)
1033+
intercept[NoSuchElementException] {
1034+
store.read(classOf[JobDataWrapper], 2)
1035+
}
1036+
}
1037+
1038+
test("eviction should respect stage completion time") {
1039+
val testConf = conf.clone().set(MAX_RETAINED_STAGES, 2)
1040+
val listener = new AppStatusListener(store, testConf, true)
1041+
1042+
val stage1 = new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1")
1043+
val stage2 = new StageInfo(2, 0, "stage2", 4, Nil, Nil, "details2")
1044+
val stage3 = new StageInfo(3, 0, "stage3", 4, Nil, Nil, "details3")
1045+
1046+
// Start stage 1 and stage 2
1047+
time += 1
1048+
stage1.submissionTime = Some(time)
1049+
listener.onStageSubmitted(SparkListenerStageSubmitted(stage1, new Properties()))
1050+
time += 1
1051+
stage2.submissionTime = Some(time)
1052+
listener.onStageSubmitted(SparkListenerStageSubmitted(stage2, new Properties()))
1053+
1054+
// Stop stage 2 before stage 1
1055+
time += 1
1056+
stage2.completionTime = Some(time)
1057+
listener.onStageCompleted(SparkListenerStageCompleted(stage2))
1058+
time += 1
1059+
stage1.completionTime = Some(time)
1060+
listener.onStageCompleted(SparkListenerStageCompleted(stage1))
1061+
1062+
// Start stage 3 and stage 2 should be evicted.
1063+
stage3.submissionTime = Some(time)
1064+
listener.onStageSubmitted(SparkListenerStageSubmitted(stage3, new Properties()))
1065+
assert(store.count(classOf[StageDataWrapper]) === 2)
1066+
intercept[NoSuchElementException] {
1067+
store.read(classOf[StageDataWrapper], Array(2, 0))
1068+
}
1069+
}
1070+
1071+
test("eviction should respect task completion time") {
1072+
val testConf = conf.clone().set(MAX_RETAINED_TASKS_PER_STAGE, 2)
1073+
val listener = new AppStatusListener(store, testConf, true)
1074+
1075+
val stage1 = new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1")
1076+
stage1.submissionTime = Some(time)
1077+
listener.onStageSubmitted(SparkListenerStageSubmitted(stage1, new Properties()))
1078+
1079+
// Start task 1 and task 2
1080+
val tasks = createTasks(3, Array("1"))
1081+
tasks.take(2).foreach { task =>
1082+
listener.onTaskStart(SparkListenerTaskStart(stage1.stageId, stage1.attemptNumber, task))
1083+
}
1084+
1085+
// Stop task 2 before task 1
1086+
time += 1
1087+
tasks(1).markFinished(TaskState.FINISHED, time)
1088+
listener.onTaskEnd(
1089+
SparkListenerTaskEnd(stage1.stageId, stage1.attemptId, "taskType", Success, tasks(1), null))
1090+
time += 1
1091+
tasks(0).markFinished(TaskState.FINISHED, time)
1092+
listener.onTaskEnd(
1093+
SparkListenerTaskEnd(stage1.stageId, stage1.attemptId, "taskType", Success, tasks(0), null))
1094+
1095+
// Start task 3 and task 2 should be evicted.
1096+
listener.onTaskStart(SparkListenerTaskStart(stage1.stageId, stage1.attemptNumber, tasks(2)))
1097+
assert(store.count(classOf[TaskDataWrapper]) === 2)
1098+
intercept[NoSuchElementException] {
1099+
store.read(classOf[TaskDataWrapper], tasks(1).id)
1100+
}
1101+
}
1102+
10131103
test("driver logs") {
10141104
val listener = new AppStatusListener(store, conf, true)
10151105

sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -332,8 +332,8 @@ class SQLAppStatusListener(
332332
return
333333
}
334334

335-
val toDelete = KVUtils.viewToSeq(kvstore.view(classOf[SQLExecutionUIData]),
336-
countToDelete.toInt) { e => e.completionTime.isDefined }
335+
val view = kvstore.view(classOf[SQLExecutionUIData]).index("completionTime").first(0L)
336+
val toDelete = KVUtils.viewToSeq(view, countToDelete.toInt)(_.completionTime.isDefined)
337337
toDelete.foreach { e => kvstore.delete(e.getClass(), e.executionId) }
338338
}
339339

sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusStore.scala

Lines changed: 7 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -23,11 +23,12 @@ import java.util.Date
2323
import scala.collection.JavaConverters._
2424
import scala.collection.mutable.ArrayBuffer
2525

26+
import com.fasterxml.jackson.annotation.JsonIgnore
2627
import com.fasterxml.jackson.databind.annotation.JsonDeserialize
2728

2829
import org.apache.spark.JobExecutionStatus
2930
import org.apache.spark.status.KVUtils.KVIndexParam
30-
import org.apache.spark.util.kvstore.KVStore
31+
import org.apache.spark.util.kvstore.{KVIndex, KVStore}
3132

3233
/**
3334
* Provides a view of a KVStore with methods that make it easy to query SQL-specific state. There's
@@ -90,7 +91,11 @@ class SQLExecutionUIData(
9091
* from the SQL listener instance.
9192
*/
9293
@JsonDeserialize(keyAs = classOf[JLong])
93-
val metricValues: Map[Long, String])
94+
val metricValues: Map[Long, String]) {
95+
96+
@JsonIgnore @KVIndex("completionTime")
97+
private def completionTimeIndex: Long = completionTime.map(_.getTime).getOrElse(-1L)
98+
}
9499

95100
class SparkPlanGraphWrapper(
96101
@KVIndexParam val executionId: Long,

sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala

Lines changed: 45 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -35,6 +35,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
3535
import org.apache.spark.sql.catalyst.util.quietly
3636
import org.apache.spark.sql.execution.{LeafExecNode, QueryExecution, SparkPlanInfo, SQLExecution}
3737
import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
38+
import org.apache.spark.sql.internal.StaticSQLConf.UI_RETAINED_EXECUTIONS
3839
import org.apache.spark.sql.test.SharedSQLContext
3940
import org.apache.spark.status.ElementTrackingStore
4041
import org.apache.spark.status.config._
@@ -510,6 +511,50 @@ class SQLAppStatusListenerSuite extends SparkFunSuite with SharedSQLContext with
510511
}
511512
}
512513

514+
test("eviction should respect execution completion time") {
515+
val conf = sparkContext.conf.clone().set(UI_RETAINED_EXECUTIONS.key, "2")
516+
val store = new ElementTrackingStore(new InMemoryStore, conf)
517+
val listener = new SQLAppStatusListener(conf, store, live = true)
518+
val statusStore = new SQLAppStatusStore(store, Some(listener))
519+
520+
var time = 0
521+
val df = createTestDataFrame
522+
// Start execution 1 and execution 2
523+
time += 1
524+
listener.onOtherEvent(SparkListenerSQLExecutionStart(
525+
1,
526+
"test",
527+
"test",
528+
df.queryExecution.toString,
529+
SparkPlanInfo.fromSparkPlan(df.queryExecution.executedPlan),
530+
time))
531+
time += 1
532+
listener.onOtherEvent(SparkListenerSQLExecutionStart(
533+
2,
534+
"test",
535+
"test",
536+
df.queryExecution.toString,
537+
SparkPlanInfo.fromSparkPlan(df.queryExecution.executedPlan),
538+
time))
539+
540+
// Stop execution 2 before execution 1
541+
time += 1
542+
listener.onOtherEvent(SparkListenerSQLExecutionEnd(2, time))
543+
time += 1
544+
listener.onOtherEvent(SparkListenerSQLExecutionEnd(1, time))
545+
546+
// Start execution 3 and execution 2 should be evicted.
547+
time += 1
548+
listener.onOtherEvent(SparkListenerSQLExecutionStart(
549+
3,
550+
"test",
551+
"test",
552+
df.queryExecution.toString,
553+
SparkPlanInfo.fromSparkPlan(df.queryExecution.executedPlan),
554+
time))
555+
assert(statusStore.executionsCount === 2)
556+
assert(statusStore.execution(2) === None)
557+
}
513558
}
514559

515560

0 commit comments

Comments
 (0)