From d379a869b85ca774ed10e78df66eef261255e8ed Mon Sep 17 00:00:00 2001 From: iRakson Date: Sat, 23 May 2020 20:02:51 +0530 Subject: [PATCH 1/2] Clean pagination code --- .../apache/spark/ui/jobs/AllJobsPage.scala | 23 +-- .../org/apache/spark/ui/jobs/StagePage.scala | 13 +- .../org/apache/spark/ui/jobs/StageTable.scala | 21 +-- .../org/apache/spark/ui/StagePageSuite.scala | 1 - .../sql/execution/ui/AllExecutionsPage.scala | 29 +--- .../thriftserver/ui/ThriftServerPage.scala | 164 ++++++++---------- 6 files changed, 93 insertions(+), 158 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala index 716d604ca31b..066512d159d0 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala @@ -214,7 +214,6 @@ private[ui] class AllJobsPage(parent: JobsTab, store: AppStatusStore) extends We val someJobHasJobGroup = jobs.exists(_.jobGroup.isDefined) val jobIdTitle = if (someJobHasJobGroup) "Job Id (Job Group)" else "Job Id" val jobPage = Option(request.getParameter(jobTag + ".page")).map(_.toInt).getOrElse(1) - val currentTime = System.currentTimeMillis() try { new JobPagedTable( @@ -226,7 +225,6 @@ private[ui] class AllJobsPage(parent: JobsTab, store: AppStatusStore) extends We UIUtils.prependBaseUri(request, parent.basePath), "jobs", // subPath killEnabled, - currentTime, jobIdTitle ).table(jobPage) } catch { @@ -399,7 +397,6 @@ private[ui] class JobDataSource( store: AppStatusStore, jobs: Seq[v1.JobData], basePath: String, - currentTime: Long, pageSize: Int, sortColumn: String, desc: Boolean) extends PagedDataSource[JobTableRowData](pageSize) { @@ -410,15 +407,9 @@ private[ui] class JobDataSource( // so that we can avoid creating duplicate contents during sorting the data private val data = jobs.map(jobRow).sorted(ordering(sortColumn, desc)) - private var _slicedJobIds: Set[Int] = null - override def dataSize: Int = data.size - override def sliceData(from: Int, to: Int): Seq[JobTableRowData] = { - val r = data.slice(from, to) - _slicedJobIds = r.map(_.jobData.jobId).toSet - r - } + override def sliceData(from: Int, to: Int): Seq[JobTableRowData] = data.slice(from, to) private def jobRow(jobData: v1.JobData): JobTableRowData = { val duration: Option[Long] = JobDataUtil.getDuration(jobData) @@ -479,17 +470,17 @@ private[ui] class JobPagedTable( basePath: String, subPath: String, killEnabled: Boolean, - currentTime: Long, jobIdTitle: String ) extends PagedTable[JobTableRowData] { + private val (sortColumn, desc, pageSize) = getTableParameters(request, jobTag, jobIdTitle) private val parameterPath = basePath + s"/$subPath/?" + getParameterOtherTable(request, jobTag) + private val encodedSortColumn = URLEncoder.encode(sortColumn, UTF_8.name()) override def tableId: String = jobTag + "-table" override def tableCssClass: String = - "table table-bordered table-sm table-striped " + - "table-head-clickable table-cell-width-limited" + "table table-bordered table-sm table-striped table-head-clickable table-cell-width-limited" override def pageSizeFormField: String = jobTag + ".pageSize" @@ -499,13 +490,11 @@ private[ui] class JobPagedTable( store, data, basePath, - currentTime, pageSize, sortColumn, desc) override def pageLink(page: Int): String = { - val encodedSortColumn = URLEncoder.encode(sortColumn, UTF_8.name()) parameterPath + s"&$pageNumberFormField=$page" + s"&$jobTag.sort=$encodedSortColumn" + @@ -514,10 +503,8 @@ private[ui] class JobPagedTable( s"#$tableHeaderId" } - override def goButtonFormPath: String = { - val encodedSortColumn = URLEncoder.encode(sortColumn, UTF_8.name()) + override def goButtonFormPath: String = s"$parameterPath&$jobTag.sort=$encodedSortColumn&$jobTag.desc=$desc#$tableHeaderId" - } override def headers: Seq[Node] = { // Information for each header: title, sortable, tooltip diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 1b072274541c..e8182a31e409 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -452,7 +452,6 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We private[ui] class TaskDataSource( stage: StageData, - currentTime: Long, pageSize: Int, sortColumn: String, desc: Boolean, @@ -474,8 +473,6 @@ private[ui] class TaskDataSource( _tasksToShow } - def tasks: Seq[TaskData] = _tasksToShow - def executorLogs(id: String): Map[String, String] = { executorIdToLogs.getOrElseUpdate(id, store.asOption(store.executorSummary(id)).map(_.executorLogs).getOrElse(Map.empty)) @@ -486,7 +483,6 @@ private[ui] class TaskDataSource( private[ui] class TaskPagedTable( stage: StageData, basePath: String, - currentTime: Long, pageSize: Int, sortColumn: String, desc: Boolean, @@ -494,6 +490,8 @@ private[ui] class TaskPagedTable( import ApiHelper._ + private val encodedSortColumn = URLEncoder.encode(sortColumn, UTF_8.name()) + override def tableId: String = "task-table" override def tableCssClass: String = @@ -505,14 +503,12 @@ private[ui] class TaskPagedTable( override val dataSource: TaskDataSource = new TaskDataSource( stage, - currentTime, pageSize, sortColumn, desc, store) override def pageLink(page: Int): String = { - val encodedSortColumn = URLEncoder.encode(sortColumn, UTF_8.name()) basePath + s"&$pageNumberFormField=$page" + s"&task.sort=$encodedSortColumn" + @@ -520,10 +516,7 @@ private[ui] class TaskPagedTable( s"&$pageSizeFormField=$pageSize" } - override def goButtonFormPath: String = { - val encodedSortColumn = URLEncoder.encode(sortColumn, UTF_8.name()) - s"$basePath&task.sort=$encodedSortColumn&task.desc=$desc" - } + override def goButtonFormPath: String = s"$basePath&task.sort=$encodedSortColumn&task.desc=$desc" def headers: Seq[Node] = { import ApiHelper._ diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index f9e84c2b2f4e..9e6eb418fe13 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -116,8 +116,7 @@ private[ui] class StagePagedTable( override def tableId: String = stageTag + "-table" override def tableCssClass: String = - "table table-bordered table-sm table-striped " + - "table-head-clickable table-cell-width-limited" + "table table-bordered table-sm table-striped table-head-clickable table-cell-width-limited" override def pageSizeFormField: String = stageTag + ".pageSize" @@ -125,7 +124,9 @@ private[ui] class StagePagedTable( private val (sortColumn, desc, pageSize) = getTableParameters(request, stageTag, "Stage Id") - val parameterPath = UIUtils.prependBaseUri(request, basePath) + s"/$subPath/?" + + private val encodedSortColumn = URLEncoder.encode(sortColumn, UTF_8.name()) + + private val parameterPath = UIUtils.prependBaseUri(request, basePath) + s"/$subPath/?" + getParameterOtherTable(request, stageTag) override val dataSource = new StageDataSource( @@ -138,7 +139,6 @@ private[ui] class StagePagedTable( ) override def pageLink(page: Int): String = { - val encodedSortColumn = URLEncoder.encode(sortColumn, UTF_8.name()) parameterPath + s"&$pageNumberFormField=$page" + s"&$stageTag.sort=$encodedSortColumn" + @@ -147,10 +147,8 @@ private[ui] class StagePagedTable( s"#$tableHeaderId" } - override def goButtonFormPath: String = { - val encodedSortColumn = URLEncoder.encode(sortColumn, UTF_8.name()) + override def goButtonFormPath: String = s"$parameterPath&$stageTag.sort=$encodedSortColumn&$stageTag.desc=$desc#$tableHeaderId" - } override def headers: Seq[Node] = { // stageHeadersAndCssClasses has three parts: header title, sortable and tooltip information. @@ -311,15 +309,9 @@ private[ui] class StageDataSource( // table so that we can avoid creating duplicate contents during sorting the data private val data = stages.map(stageRow).sorted(ordering(sortColumn, desc)) - private var _slicedStageIds: Set[Int] = _ - override def dataSize: Int = data.size - override def sliceData(from: Int, to: Int): Seq[StageTableRowData] = { - val r = data.slice(from, to) - _slicedStageIds = r.map(_.stageId).toSet - r - } + override def sliceData(from: Int, to: Int): Seq[StageTableRowData] = data.slice(from, to) private def stageRow(stageData: v1.StageData): StageTableRowData = { val formattedSubmissionTime = stageData.submissionTime match { @@ -350,7 +342,6 @@ private[ui] class StageDataSource( val shuffleWrite = stageData.shuffleWriteBytes val shuffleWriteWithUnit = if (shuffleWrite > 0) Utils.bytesToString(shuffleWrite) else "" - new StageTableRowData( stageData, Some(stageData), diff --git a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala index 5d34a5647337..3d52199b0132 100644 --- a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala @@ -98,7 +98,6 @@ class StagePageSuite extends SparkFunSuite with LocalSparkContext { val taskTable = new TaskPagedTable( stageData, basePath = "/a/b/c", - currentTime = 0, pageSize = 10, sortColumn = "Index", desc = false, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala index 8c23f2cbb86b..33539c01ee5d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala @@ -203,11 +203,10 @@ private[ui] class ExecutionPagedTable( private val (sortColumn, desc, pageSize) = getTableParameters(request, executionTag, "ID") + private val encodedSortColumn = URLEncoder.encode(sortColumn, UTF_8.name()) + override val dataSource = new ExecutionDataSource( - request, - parent, data, - basePath, currentTime, pageSize, sortColumn, @@ -222,11 +221,9 @@ private[ui] class ExecutionPagedTable( override def tableId: String = s"$executionTag-table" override def tableCssClass: String = - "table table-bordered table-sm table-striped " + - "table-head-clickable table-cell-width-limited" + "table table-bordered table-sm table-striped table-head-clickable table-cell-width-limited" override def pageLink(page: Int): String = { - val encodedSortColumn = URLEncoder.encode(sortColumn, UTF_8.name()) parameterPath + s"&$pageNumberFormField=$page" + s"&$executionTag.sort=$encodedSortColumn" + @@ -239,10 +236,8 @@ private[ui] class ExecutionPagedTable( override def pageNumberFormField: String = s"$executionTag.page" - override def goButtonFormPath: String = { - val encodedSortColumn = URLEncoder.encode(sortColumn, UTF_8.name()) + override def goButtonFormPath: String = s"$parameterPath&$executionTag.sort=$encodedSortColumn&$executionTag.desc=$desc#$tableHeaderId" - } override def headers: Seq[Node] = { // Information for each header: title, sortable, tooltip @@ -348,7 +343,6 @@ private[ui] class ExecutionPagedTable( private[ui] class ExecutionTableRowData( - val submissionTime: Long, val duration: Long, val executionUIData: SQLExecutionUIData, val runningJobData: Seq[Int], @@ -357,10 +351,7 @@ private[ui] class ExecutionTableRowData( private[ui] class ExecutionDataSource( - request: HttpServletRequest, - parent: SQLTab, executionData: Seq[SQLExecutionUIData], - basePath: String, currentTime: Long, pageSize: Int, sortColumn: String, @@ -373,20 +364,13 @@ private[ui] class ExecutionDataSource( // in the table so that we can avoid creating duplicate contents during sorting the data private val data = executionData.map(executionRow).sorted(ordering(sortColumn, desc)) - private var _sliceExecutionIds: Set[Int] = _ - override def dataSize: Int = data.size - override def sliceData(from: Int, to: Int): Seq[ExecutionTableRowData] = { - val r = data.slice(from, to) - _sliceExecutionIds = r.map(_.executionUIData.executionId.toInt).toSet - r - } + override def sliceData(from: Int, to: Int): Seq[ExecutionTableRowData] = data.slice(from, to) private def executionRow(executionUIData: SQLExecutionUIData): ExecutionTableRowData = { - val submissionTime = executionUIData.submissionTime val duration = executionUIData.completionTime.map(_.getTime()) - .getOrElse(currentTime) - submissionTime + .getOrElse(currentTime) - executionUIData.submissionTime val runningJobData = if (showRunningJobs) { executionUIData.jobs.filter { @@ -407,7 +391,6 @@ private[ui] class ExecutionDataSource( } else Seq.empty new ExecutionTableRowData( - submissionTime, duration, executionUIData, runningJobData, diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala index 1f2d4b1b8777..8efbdb30c605 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala @@ -175,19 +175,19 @@ private[ui] class SqlStatsPagedTable( private val (sortColumn, desc, pageSize) = getTableParameters(request, sqlStatsTableTag, "Start Time") - override val dataSource = new SqlStatsTableDataSource(data, pageSize, sortColumn, desc) + private val encodedSortColumn = URLEncoder.encode(sortColumn, UTF_8.name()) private val parameterPath = s"$basePath/$subPath/?${getParameterOtherTable(request, sqlStatsTableTag)}" + override val dataSource = new SqlStatsTableDataSource(data, pageSize, sortColumn, desc) + override def tableId: String = sqlStatsTableTag override def tableCssClass: String = - "table table-bordered table-sm table-striped " + - "table-head-clickable table-cell-width-limited" + "table table-bordered table-sm table-striped table-head-clickable table-cell-width-limited" override def pageLink(page: Int): String = { - val encodedSortColumn = URLEncoder.encode(sortColumn, UTF_8.name()) parameterPath + s"&$pageNumberFormField=$page" + s"&$sqlStatsTableTag.sort=$encodedSortColumn" + @@ -200,11 +200,9 @@ private[ui] class SqlStatsPagedTable( override def pageNumberFormField: String = s"$sqlStatsTableTag.page" - override def goButtonFormPath: String = { - val encodedSortColumn = URLEncoder.encode(sortColumn, UTF_8.name()) + override def goButtonFormPath: String = s"$parameterPath&$sqlStatsTableTag.sort=$encodedSortColumn" + s"&$sqlStatsTableTag.desc=$desc#$sqlStatsTableTag" - } override def headers: Seq[Node] = { val sqlTableHeadersAndTooltips: Seq[(String, Boolean, Option[String])] = @@ -307,19 +305,19 @@ private[ui] class SessionStatsPagedTable( private val (sortColumn, desc, pageSize) = getTableParameters(request, sessionStatsTableTag, "Start Time") - override val dataSource = new SessionStatsTableDataSource(data, pageSize, sortColumn, desc) + private val encodedSortColumn = URLEncoder.encode(sortColumn, UTF_8.name()) private val parameterPath = s"$basePath/$subPath/?${getParameterOtherTable(request, sessionStatsTableTag)}" + override val dataSource = new SessionStatsTableDataSource(data, pageSize, sortColumn, desc) + override def tableId: String = sessionStatsTableTag override def tableCssClass: String = - "table table-bordered table-sm table-striped " + - "table-head-clickable table-cell-width-limited" + "table table-bordered table-sm table-striped table-head-clickable table-cell-width-limited" override def pageLink(page: Int): String = { - val encodedSortColumn = URLEncoder.encode(sortColumn, UTF_8.name()) parameterPath + s"&$pageNumberFormField=$page" + s"&$sessionStatsTableTag.sort=$encodedSortColumn" + @@ -332,11 +330,9 @@ private[ui] class SessionStatsPagedTable( override def pageNumberFormField: String = s"$sessionStatsTableTag.page" - override def goButtonFormPath: String = { - val encodedSortColumn = URLEncoder.encode(sortColumn, UTF_8.name()) + override def goButtonFormPath: String = s"$parameterPath&$sessionStatsTableTag.sort=$encodedSortColumn" + s"&$sessionStatsTableTag.desc=$desc#$sessionStatsTableTag" - } override def headers: Seq[Node] = { val sessionTableHeadersAndTooltips: Seq[(String, Boolean, Option[String])] = @@ -370,108 +366,94 @@ private[ui] class SessionStatsPagedTable( } } - private[ui] class SqlStatsTableRow( +private[ui] class SqlStatsTableRow( val jobId: Seq[String], val duration: Long, val executionTime: Long, val executionInfo: ExecutionInfo, val detail: String) - private[ui] class SqlStatsTableDataSource( +private[ui] class SqlStatsTableDataSource( info: Seq[ExecutionInfo], pageSize: Int, sortColumn: String, desc: Boolean) extends PagedDataSource[SqlStatsTableRow](pageSize) { - // Convert ExecutionInfo to SqlStatsTableRow which contains the final contents to show in - // the table so that we can avoid creating duplicate contents during sorting the data - private val data = info.map(sqlStatsTableRow).sorted(ordering(sortColumn, desc)) - - private var _slicedStartTime: Set[Long] = null + // Convert ExecutionInfo to SqlStatsTableRow which contains the final contents to show in + // the table so that we can avoid creating duplicate contents during sorting the data + private val data = info.map(sqlStatsTableRow).sorted(ordering(sortColumn, desc)) - override def dataSize: Int = data.size + override def dataSize: Int = data.size - override def sliceData(from: Int, to: Int): Seq[SqlStatsTableRow] = { - val r = data.slice(from, to) - _slicedStartTime = r.map(_.executionInfo.startTimestamp).toSet - r - } + override def sliceData(from: Int, to: Int): Seq[SqlStatsTableRow] = data.slice(from, to) - private def sqlStatsTableRow(executionInfo: ExecutionInfo): SqlStatsTableRow = { - val duration = executionInfo.totalTime(executionInfo.closeTimestamp) - val executionTime = executionInfo.totalTime(executionInfo.finishTimestamp) - val detail = Option(executionInfo.detail).filter(!_.isEmpty) - .getOrElse(executionInfo.executePlan) - val jobId = executionInfo.jobId.toSeq.sorted + private def sqlStatsTableRow(executionInfo: ExecutionInfo): SqlStatsTableRow = { + val duration = executionInfo.totalTime(executionInfo.closeTimestamp) + val executionTime = executionInfo.totalTime(executionInfo.finishTimestamp) + val detail = Option(executionInfo.detail).filter(!_.isEmpty) + .getOrElse(executionInfo.executePlan) + val jobId = executionInfo.jobId.toSeq.sorted - new SqlStatsTableRow(jobId, duration, executionTime, executionInfo, detail) + new SqlStatsTableRow(jobId, duration, executionTime, executionInfo, detail) + } + /** + * Return Ordering according to sortColumn and desc. + */ + private def ordering(sortColumn: String, desc: Boolean): Ordering[SqlStatsTableRow] = { + val ordering: Ordering[SqlStatsTableRow] = sortColumn match { + case "User" => Ordering.by(_.executionInfo.userName) + case "JobID" => Ordering by (_.jobId.headOption) + case "GroupID" => Ordering.by(_.executionInfo.groupId) + case "Start Time" => Ordering.by(_.executionInfo.startTimestamp) + case "Finish Time" => Ordering.by(_.executionInfo.finishTimestamp) + case "Close Time" => Ordering.by(_.executionInfo.closeTimestamp) + case "Execution Time" => Ordering.by(_.executionTime) + case "Duration" => Ordering.by(_.duration) + case "Statement" => Ordering.by(_.executionInfo.statement) + case "State" => Ordering.by(_.executionInfo.state) + case "Detail" => Ordering.by(_.detail) + case unknownColumn => throw new IllegalArgumentException(s"Unknown column: $unknownColumn") } - - /** - * Return Ordering according to sortColumn and desc. - */ - private def ordering(sortColumn: String, desc: Boolean): Ordering[SqlStatsTableRow] = { - val ordering: Ordering[SqlStatsTableRow] = sortColumn match { - case "User" => Ordering.by(_.executionInfo.userName) - case "JobID" => Ordering by (_.jobId.headOption) - case "GroupID" => Ordering.by(_.executionInfo.groupId) - case "Start Time" => Ordering.by(_.executionInfo.startTimestamp) - case "Finish Time" => Ordering.by(_.executionInfo.finishTimestamp) - case "Close Time" => Ordering.by(_.executionInfo.closeTimestamp) - case "Execution Time" => Ordering.by(_.executionTime) - case "Duration" => Ordering.by(_.duration) - case "Statement" => Ordering.by(_.executionInfo.statement) - case "State" => Ordering.by(_.executionInfo.state) - case "Detail" => Ordering.by(_.detail) - case unknownColumn => throw new IllegalArgumentException(s"Unknown column: $unknownColumn") - } - if (desc) { - ordering.reverse - } else { - ordering - } + if (desc) { + ordering.reverse + } else { + ordering } - } +} - private[ui] class SessionStatsTableDataSource( +private[ui] class SessionStatsTableDataSource( info: Seq[SessionInfo], pageSize: Int, sortColumn: String, desc: Boolean) extends PagedDataSource[SessionInfo](pageSize) { - // Sorting SessionInfo data - private val data = info.sorted(ordering(sortColumn, desc)) - - private var _slicedStartTime: Set[Long] = null - - override def dataSize: Int = data.size - - override def sliceData(from: Int, to: Int): Seq[SessionInfo] = { - val r = data.slice(from, to) - _slicedStartTime = r.map(_.startTimestamp).toSet - r + // Sorting SessionInfo data + private val data = info.sorted(ordering(sortColumn, desc)) + + override def dataSize: Int = data.size + + override def sliceData(from: Int, to: Int): Seq[SessionInfo] = data.slice(from, to) + + /** + * Return Ordering according to sortColumn and desc. + */ + private def ordering(sortColumn: String, desc: Boolean): Ordering[SessionInfo] = { + val ordering: Ordering[SessionInfo] = sortColumn match { + case "User" => Ordering.by(_.userName) + case "IP" => Ordering.by(_.ip) + case "Session ID" => Ordering.by(_.sessionId) + case "Start Time" => Ordering by (_.startTimestamp) + case "Finish Time" => Ordering.by(_.finishTimestamp) + case "Duration" => Ordering.by(_.totalTime) + case "Total Execute" => Ordering.by(_.totalExecution) + case unknownColumn => throw new IllegalArgumentException(s"Unknown column: $unknownColumn") } - - /** - * Return Ordering according to sortColumn and desc. - */ - private def ordering(sortColumn: String, desc: Boolean): Ordering[SessionInfo] = { - val ordering: Ordering[SessionInfo] = sortColumn match { - case "User" => Ordering.by(_.userName) - case "IP" => Ordering.by(_.ip) - case "Session ID" => Ordering.by(_.sessionId) - case "Start Time" => Ordering by (_.startTimestamp) - case "Finish Time" => Ordering.by(_.finishTimestamp) - case "Duration" => Ordering.by(_.totalTime) - case "Total Execute" => Ordering.by(_.totalExecution) - case unknownColumn => throw new IllegalArgumentException(s"Unknown column: $unknownColumn") - } - if (desc) { - ordering.reverse - } else { - ordering - } + if (desc) { + ordering.reverse + } else { + ordering } } +} From 65007e75e7f3bd46539be54a1bb3b058c160c9c4 Mon Sep 17 00:00:00 2001 From: iRakson Date: Sat, 23 May 2020 20:16:43 +0530 Subject: [PATCH 2/2] fix --- core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index e8182a31e409..47ba951953ce 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -212,7 +212,6 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We stageData, UIUtils.prependBaseUri(request, parent.basePath) + s"/stages/stage/?id=${stageId}&attempt=${stageAttemptId}", - currentTime, pageSize = taskPageSize, sortColumn = taskSortColumn, desc = taskSortDesc,