From 5450258d7aef0772b9406c0ea5abb51096642ebb Mon Sep 17 00:00:00 2001 From: pgandhi Date: Mon, 4 Jun 2018 16:00:06 -0500 Subject: [PATCH 01/42] [SPARK-21809] : Changing file executorspage-template.html and executorspage.js --- .../ui/static/executorspage-template.html | 4 +- .../apache/spark/ui/static/executorspage.js | 60 +------------------ 2 files changed, 5 insertions(+), 59 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html b/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html index 5c91304e49fd7..16806d9af9095 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html @@ -16,7 +16,7 @@ --> diff --git a/core/src/main/resources/org/apache/spark/ui/static/utils.js b/core/src/main/resources/org/apache/spark/ui/static/utils.js index 4f63f6413d6de..f53d8ae6adef0 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/utils.js +++ b/core/src/main/resources/org/apache/spark/ui/static/utils.js @@ -74,3 +74,92 @@ function getTimeZone() { return new Date().toString().match(/\((.*)\)/)[1]; } } + +function formatLogsCells(execLogs, type) { + if (type !== 'display') return Object.keys(execLogs); + if (!execLogs) return; + var result = ''; + $.each(execLogs, function (logName, logUrl) { + result += '
' + logName + '
' + }); + return result; +} + +function getStandAloneAppId(cb) { + var words = document.baseURI.split('/'); + var ind = words.indexOf("proxy"); + if (ind > 0) { + var appId = words[ind + 1]; + cb(appId); + return; + } + ind = words.indexOf("history"); + if (ind > 0) { + var appId = words[ind + 1]; + cb(appId); + return; + } + //Looks like Web UI is running in standalone mode + //Let's get application-id using REST End Point + $.getJSON(location.origin + "/api/v1/applications", function(response, status, jqXHR) { + if (response && response.length > 0) { + var appId = response[0].id; + cb(appId); + return; + } + }); +} + +// This function is a helper function for sorting in datatable. +// When the data is in duration (e.g. 12ms 2s 2min 2h ) +// It will convert the string into integer for correct ordering +function ConvertDurationString(data) { + data = data.toString(); + var units = data.replace(/[\d\.]/g, '' ) + .replace(' ', '') + .toLowerCase(); + var multiplier = 1; + + switch(units) { + case 's': + multiplier = 1000; + break; + case 'min': + multiplier = 600000; + break; + case 'h': + multiplier = 3600000; + break; + default: + break; + } + return parseFloat(data) * multiplier; +} + +function createTemplateURI(appId, templateName) { + var words = document.baseURI.split('/'); + var ind = words.indexOf("proxy"); + if (ind > 0) { + var baseURI = words.slice(0, ind + 1).join('/') + '/' + appId + '/static/' + templateName + '-template.html'; + return baseURI; + } + ind = words.indexOf("history"); + if(ind > 0) { + var baseURI = words.slice(0, ind).join('/') + '/static/' + templateName + '-template.html'; + return baseURI; + } + return location.origin + "/static/" + templateName + "-template.html"; +} + +function setDataTableDefaults() { + $.extend($.fn.dataTable.defaults, { + stateSave: true, + lengthMenu: [[20, 40, 60, 100, -1], [20, 40, 60, 100, "All"]], + pageLength: 20 + }); +} + +function formatDate(date) { + if (date <= 0) return "-"; + else return date.split(".")[0].replace("T", " "); +} diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index 935d9b1aec615..29739456b603f 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.css +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css @@ -251,4 +251,13 @@ a.expandbutton { .table-cell-width-limited td { max-width: 600px; +} + +.title-table { + clear: left; + display: inline-block; +} + +.table-dataTable{ + width="100%"; } \ No newline at end of file From fc88decce9ac37ec3c639cda722464f26727e1be Mon Sep 17 00:00:00 2001 From: pgandhi Date: Fri, 15 Jun 2018 14:30:57 -0500 Subject: [PATCH 03/42] [SPARK-21809] : Getting rquired data in api endpoints --- .../org/apache/spark/ui/static/stagepage.js | 9 +- .../org/apache/spark/status/LiveEntity.scala | 4 +- .../spark/status/api/v1/StagesResource.scala | 21 +- .../org/apache/spark/status/api/v1/api.scala | 13 +- .../org/apache/spark/status/storeTypes.scala | 5 +- .../apache/spark/ui/jobs/ExecutorTable.scala | 152 ---------- .../org/apache/spark/ui/jobs/StagePage.scala | 274 +----------------- .../spark/status/AppStatusUtilsSuite.scala | 10 +- 8 files changed, 61 insertions(+), 427 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala diff --git a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js index 5a9813d8d4ad2..05f6024fff6dd 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js @@ -109,10 +109,11 @@ $(document).ready(function () { $.getJSON(endPoint, function(response, status, jqXHR) { // prepare data for tasks table + console.log("hereeeeeeeeee 1 "+JSON.stringify(response[0].tasks[0])); var indices = Object.keys(response[0].tasks); var task_table = []; indices.forEach(function (ix) { - task_table.push(response[0].tasks[ix]); + task_table.push(response[0].tasks[parseInt(ix)]); }); // prepare data for task aggregated metrics table @@ -249,7 +250,7 @@ $(document).ready(function () { "columns": [ {data : "id"}, {data : "executorLogs", render: formatLogsCells}, - {data : "host"}, + {data : "hostPort"}, { data : function (row, type) { return type === 'display' ? formatDuration(row.taskTime) : row.taskTime; @@ -264,7 +265,7 @@ $(document).ready(function () { {data : "failedTasks"}, {data : "killedTasks"}, {data : "succeededTasks"}, - {data : "blacklisted"}, + {data : "isBlacklistedForStage"}, { data : function (row, type) { return row.inputRecords != 0 ? formatBytes(row.inputBytes/row.inputRecords) : 0; @@ -327,7 +328,7 @@ $(document).ready(function () { }, {data : "taskId"}, {data : "attempt"}, - {data : "taskState"}, + {data : "status"}, {data : "taskLocality"}, { data : function (row, type) { diff --git a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala index 79e3f13b826ce..2e8cc7dd4fe67 100644 --- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala +++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala @@ -339,7 +339,9 @@ private class LiveExecutorStageSummary( metrics.shuffleWriteMetrics.recordsWritten, metrics.memoryBytesSpilled, metrics.diskBytesSpilled, - isBlacklisted) + isBlacklisted, + null, + null) new ExecutorStageSummaryWrapper(stageId, attemptId, executorId, info) } diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala index 96249e4bfd5fa..288188b7d84eb 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala @@ -22,6 +22,7 @@ import javax.ws.rs.core.MediaType import org.apache.spark.SparkException import org.apache.spark.scheduler.StageInfo +import org.apache.spark.status.AppStatusUtils import org.apache.spark.status.api.v1.StageStatus._ import org.apache.spark.status.api.v1.TaskSorting._ import org.apache.spark.ui.SparkUI @@ -40,8 +41,26 @@ private[v1] class StagesResource extends BaseAppResource { @PathParam("stageId") stageId: Int, @QueryParam("details") @DefaultValue("true") details: Boolean): Seq[StageData] = { withUI { ui => - val ret = ui.store.stageData(stageId, details = details) + var ret = ui.store.stageData(stageId, details = details) if (ret.nonEmpty) { + for (i <- 0 to (ret.length - 1)) { + val executorIdArray = ret(i).executorSummary.get.keys.toArray + for (execId <- executorIdArray) { + val executorLogs = ui.store.executorSummary(execId).executorLogs + val hostPort = ui.store.executorSummary(execId).hostPort + val taskDataArray = ret(i).tasks.get.keys.toArray + val executorStageSummaryArray = ret(i).executorSummary.get.keys.toArray + for (taskData <- taskDataArray) { + ret(i).tasks.get.get(taskData).get.executorLogs = executorLogs + ret(i).tasks.get.get(taskData).get.schedulerDelay = AppStatusUtils.schedulerDelay(ret(i).tasks.get.get(taskData).get) + ret(i).tasks.get.get(taskData).get.gettingResultTime = AppStatusUtils.gettingResultTime(ret(i).tasks.get.get(taskData).get) + } + for (executorStageSummary <- executorStageSummaryArray) { + ret(i).executorSummary.get.get(executorStageSummary).get.executorLogs = executorLogs + ret(i).executorSummary.get.get(executorStageSummary).get.hostPort = hostPort + } + } + } ret } else { throw new NotFoundException(s"unknown stage: $stageId") diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala index 971d7e90fa7b8..9678aeab7ca54 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala @@ -71,7 +71,9 @@ class ExecutorStageSummary private[spark]( val shuffleWriteRecords : Long, val memoryBytesSpilled : Long, val diskBytesSpilled : Long, - val isBlacklistedForStage: Boolean) + val isBlacklistedForStage: Boolean, + var executorLogs: Map[String, String], + var hostPort: String) class ExecutorSummary private[spark]( val id: String, @@ -196,8 +198,8 @@ class StageData private[spark]( val rddIds: Seq[Int], val accumulatorUpdates: Seq[AccumulableInfo], - val tasks: Option[Map[Long, TaskData]], - val executorSummary: Option[Map[String, ExecutorStageSummary]], + var tasks: Option[Map[Long, TaskData]], + var executorSummary: Option[Map[String, ExecutorStageSummary]], val killedTasksSummary: Map[String, Int]) class TaskData private[spark]( @@ -215,7 +217,10 @@ class TaskData private[spark]( val speculative: Boolean, val accumulatorUpdates: Seq[AccumulableInfo], val errorMessage: Option[String] = None, - val taskMetrics: Option[TaskMetrics] = None) + val taskMetrics: Option[TaskMetrics] = None, + var executorLogs: Map[String, String], + var schedulerDelay: Long, + var gettingResultTime: Long) class TaskMetrics private[spark]( val executorDeserializeTime: Long, diff --git a/core/src/main/scala/org/apache/spark/status/storeTypes.scala b/core/src/main/scala/org/apache/spark/status/storeTypes.scala index 646cf25880e37..faf21d209fc45 100644 --- a/core/src/main/scala/org/apache/spark/status/storeTypes.scala +++ b/core/src/main/scala/org/apache/spark/status/storeTypes.scala @@ -283,7 +283,10 @@ private[spark] class TaskDataWrapper( speculative, accumulatorUpdates, errorMessage, - metrics) + metrics, + null, + 0L, + 0L) } @JsonIgnore @KVIndex(TaskIndexNames.STAGE) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala deleted file mode 100644 index 0ff64f053f371..0000000000000 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ /dev/null @@ -1,152 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.ui.jobs - -import scala.xml.{Node, Unparsed} - -import org.apache.spark.status.AppStatusStore -import org.apache.spark.status.api.v1.StageData -import org.apache.spark.ui.{ToolTips, UIUtils} -import org.apache.spark.util.Utils - -/** Stage summary grouped by executors. */ -private[ui] class ExecutorTable(stage: StageData, store: AppStatusStore) { - - import ApiHelper._ - - def toNodeSeq: Seq[Node] = { - - - - - - - - - - {if (hasInput(stage)) { - - }} - {if (hasOutput(stage)) { - - }} - {if (hasShuffleRead(stage)) { - - }} - {if (hasShuffleWrite(stage)) { - - }} - {if (hasBytesSpilled(stage)) { - - - }} - - - - {createExecutorTable(stage)} - -
Executor IDAddressTask TimeTotal TasksFailed TasksKilled TasksSucceeded Tasks - Input Size / Records - - Output Size / Records - - - Shuffle Read Size / Records - - - Shuffle Write Size / Records - Shuffle Spill (Memory)Shuffle Spill (Disk) - - Blacklisted - -
- - } - - private def createExecutorTable(stage: StageData) : Seq[Node] = { - val executorSummary = store.executorSummary(stage.stageId, stage.attemptId) - - executorSummary.toSeq.sortBy(_._1).map { case (k, v) => - val executor = store.asOption(store.executorSummary(k)) - - -
{k}
-
- { - executor.map(_.executorLogs).getOrElse(Map.empty).map { - case (logName, logUrl) =>
{logName}
- } - } -
- - {executor.map { e => e.hostPort }.getOrElse("CANNOT FIND ADDRESS")} - {UIUtils.formatDuration(v.taskTime)} - {v.failedTasks + v.succeededTasks + v.killedTasks} - {v.failedTasks} - {v.killedTasks} - {v.succeededTasks} - {if (hasInput(stage)) { - - {s"${Utils.bytesToString(v.inputBytes)} / ${v.inputRecords}"} - - }} - {if (hasOutput(stage)) { - - {s"${Utils.bytesToString(v.outputBytes)} / ${v.outputRecords}"} - - }} - {if (hasShuffleRead(stage)) { - - {s"${Utils.bytesToString(v.shuffleRead)} / ${v.shuffleReadRecords}"} - - }} - {if (hasShuffleWrite(stage)) { - - {s"${Utils.bytesToString(v.shuffleWrite)} / ${v.shuffleWriteRecords}"} - - }} - {if (hasBytesSpilled(stage)) { - - {Utils.bytesToString(v.memoryBytesSpilled)} - - - {Utils.bytesToString(v.diskBytesSpilled)} - - }} - { - if (executor.map(_.isBlacklisted).getOrElse(false)) { - for application - } else if (v.isBlacklistedForStage) { - for stage - } else { - false - } - } - - } - } - -} 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 d4e6a7bc3effa..1c73698127eff 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 @@ -185,89 +185,9 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We - val showAdditionalMetrics = -
- - - Show Additional Metrics - - -
- val stageGraph = parent.store.asOption(parent.store.operationGraphForStage(stageId)) val dagViz = UIUtils.showDagVizForStage(stageId, stageGraph) - val accumulableHeaders: Seq[String] = Seq("Accumulable", "Value") - def accumulableRow(acc: AccumulableInfo): Seq[Node] = { - if (acc.name != null && acc.value != null) { - {acc.name}{acc.value} - } else { - Nil - } - } - val accumulableTable = UIUtils.listingTable( - accumulableHeaders, - accumulableRow, - stageData.accumulatorUpdates.toSeq) - val page: Int = { // If the user has changed to a larger page size, then go to page 1 in order to avoid // IndexOutOfBoundsException. @@ -278,7 +198,7 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We } } val currentTime = System.currentTimeMillis() - val (taskTable, taskTableHTML) = try { + val taskTable = try { val _taskTable = new TaskPagedTable( stageData, UIUtils.prependBaseUri(request, parent.basePath) + @@ -289,17 +209,11 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We desc = taskSortDesc, store = parent.store ) - (_taskTable, _taskTable.table(page)) + _taskTable.table(page) + _taskTable } catch { case e @ (_ : IllegalArgumentException | _ : IndexOutOfBoundsException) => - val errorMessage = -
-

Error while rendering stage table:

-
-              {Utils.exceptionString(e)}
-            
-
- (null, errorMessage) + null } val jsForScrollingDownToTaskTable = @@ -320,184 +234,20 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We val metricsSummary = store.taskSummary(stageData.stageId, stageData.attemptId, Array(0, 0.25, 0.5, 0.75, 1.0)) - val summaryTable = metricsSummary.map { metrics => - def timeQuantiles(data: IndexedSeq[Double]): Seq[Node] = { - data.map { millis => - {UIUtils.formatDuration(millis.toLong)} - } - } - - def sizeQuantiles(data: IndexedSeq[Double]): Seq[Node] = { - data.map { size => - {Utils.bytesToString(size.toLong)} - } - } - - def sizeQuantilesWithRecords( - data: IndexedSeq[Double], - records: IndexedSeq[Double]) : Seq[Node] = { - data.zip(records).map { case (d, r) => - {s"${Utils.bytesToString(d.toLong)} / ${r.toLong}"} - } - } - - def titleCell(title: String, tooltip: String): Seq[Node] = { - - - {title} - - - } - - def simpleTitleCell(title: String): Seq[Node] = {title} - - val deserializationQuantiles = titleCell("Task Deserialization Time", - ToolTips.TASK_DESERIALIZATION_TIME) ++ timeQuantiles(metrics.executorDeserializeTime) - - val serviceQuantiles = simpleTitleCell("Duration") ++ timeQuantiles(metrics.executorRunTime) - - val gcQuantiles = titleCell("GC Time", ToolTips.GC_TIME) ++ timeQuantiles(metrics.jvmGcTime) - - val serializationQuantiles = titleCell("Result Serialization Time", - ToolTips.RESULT_SERIALIZATION_TIME) ++ timeQuantiles(metrics.resultSerializationTime) - - val gettingResultQuantiles = titleCell("Getting Result Time", ToolTips.GETTING_RESULT_TIME) ++ - timeQuantiles(metrics.gettingResultTime) - - val peakExecutionMemoryQuantiles = titleCell("Peak Execution Memory", - ToolTips.PEAK_EXECUTION_MEMORY) ++ sizeQuantiles(metrics.peakExecutionMemory) - - // The scheduler delay includes the network delay to send the task to the worker - // machine and to send back the result (but not the time to fetch the task result, - // if it needed to be fetched from the block manager on the worker). - val schedulerDelayQuantiles = titleCell("Scheduler Delay", ToolTips.SCHEDULER_DELAY) ++ - timeQuantiles(metrics.schedulerDelay) - - def inputQuantiles: Seq[Node] = { - simpleTitleCell("Input Size / Records") ++ - sizeQuantilesWithRecords(metrics.inputMetrics.bytesRead, metrics.inputMetrics.recordsRead) - } - - def outputQuantiles: Seq[Node] = { - simpleTitleCell("Output Size / Records") ++ - sizeQuantilesWithRecords(metrics.outputMetrics.bytesWritten, - metrics.outputMetrics.recordsWritten) - } - - def shuffleReadBlockedQuantiles: Seq[Node] = { - titleCell("Shuffle Read Blocked Time", ToolTips.SHUFFLE_READ_BLOCKED_TIME) ++ - timeQuantiles(metrics.shuffleReadMetrics.fetchWaitTime) - } - - def shuffleReadTotalQuantiles: Seq[Node] = { - titleCell("Shuffle Read Size / Records", ToolTips.SHUFFLE_READ) ++ - sizeQuantilesWithRecords(metrics.shuffleReadMetrics.readBytes, - metrics.shuffleReadMetrics.readRecords) - } - - def shuffleReadRemoteQuantiles: Seq[Node] = { - titleCell("Shuffle Remote Reads", ToolTips.SHUFFLE_READ_REMOTE_SIZE) ++ - sizeQuantiles(metrics.shuffleReadMetrics.remoteBytesRead) - } - - def shuffleWriteQuantiles: Seq[Node] = { - simpleTitleCell("Shuffle Write Size / Records") ++ - sizeQuantilesWithRecords(metrics.shuffleWriteMetrics.writeBytes, - metrics.shuffleWriteMetrics.writeRecords) - } - - def memoryBytesSpilledQuantiles: Seq[Node] = { - simpleTitleCell("Shuffle spill (memory)") ++ sizeQuantiles(metrics.memoryBytesSpilled) - } - - def diskBytesSpilledQuantiles: Seq[Node] = { - simpleTitleCell("Shuffle spill (disk)") ++ sizeQuantiles(metrics.diskBytesSpilled) - } - - val listings: Seq[Seq[Node]] = Seq( - {serviceQuantiles}, - {schedulerDelayQuantiles}, - - {deserializationQuantiles} - - {gcQuantiles}, - - {serializationQuantiles} - , - {gettingResultQuantiles}, - - {peakExecutionMemoryQuantiles} - , - if (hasInput(stageData)) {inputQuantiles} else Nil, - if (hasOutput(stageData)) {outputQuantiles} else Nil, - if (hasShuffleRead(stageData)) { - - {shuffleReadBlockedQuantiles} - - {shuffleReadTotalQuantiles} - - {shuffleReadRemoteQuantiles} - - } else { - Nil - }, - if (hasShuffleWrite(stageData)) {shuffleWriteQuantiles} else Nil, - if (hasBytesSpilled(stageData)) {memoryBytesSpilledQuantiles} else Nil, - if (hasBytesSpilled(stageData)) {diskBytesSpilledQuantiles} else Nil) - - val quantileHeaders = Seq("Metric", "Min", "25th percentile", "Median", "75th percentile", - "Max") - // The summary table does not use CSS to stripe rows, which doesn't work with hidden - // rows (instead, JavaScript in table.js is used to stripe the non-hidden rows). - UIUtils.listingTable( - quantileHeaders, - identity[Seq[Node]], - listings, - fixedWidth = true, - id = Some("task-summary-table"), - stripeRowsWithCss = false) - } - - val executorTable = new ExecutorTable(stageData, parent.store) - - val maybeAccumulableTable: Seq[Node] = - if (hasAccumulators(stageData)) {

Accumulators

++ accumulableTable } else Seq() - - val aggMetrics = - -

- - Aggregated Metrics by Executor -

-
-
- {executorTable.toNodeSeq} -
- val content = summary ++ - dagViz ++ - showAdditionalMetrics ++ + dagViz ++
++ makeTimeline( // Only show the tasks in the table Option(taskTable).map(_.dataSource.tasks).getOrElse(Nil), currentTime) ++ -

Summary Metrics for {numCompleted} Completed Tasks

++ -
{summaryTable.getOrElse("No tasks have reported metrics yet.")}
++ - aggMetrics ++ - maybeAccumulableTable ++ - -

- - Tasks ({totalTasksNumStr}) -

-
++ -
- {taskTableHTML ++ jsForScrollingDownToTaskTable} -
- UIUtils.headerSparkPage(request, stageHeader, content, parent, showVisualization = true) +
+ + +
+ UIUtils.headerSparkPage(request, stageHeader, content, parent, showVisualization = true, + useDataTables = true) + } def makeTimeline(tasks: Seq[TaskData], currentTime: Long): Seq[Node] = { diff --git a/core/src/test/scala/org/apache/spark/status/AppStatusUtilsSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStatusUtilsSuite.scala index 9e74e86ad54b9..a01b24d323d28 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusUtilsSuite.scala @@ -52,7 +52,10 @@ class AppStatusUtilsSuite extends SparkFunSuite { inputMetrics = null, outputMetrics = null, shuffleReadMetrics = null, - shuffleWriteMetrics = null))) + shuffleWriteMetrics = null)), + executorLogs = null, + schedulerDelay = 0L, + gettingResultTime = 0L) assert(AppStatusUtils.schedulerDelay(runningTask) === 0L) val finishedTask = new TaskData( @@ -83,7 +86,10 @@ class AppStatusUtilsSuite extends SparkFunSuite { inputMetrics = null, outputMetrics = null, shuffleReadMetrics = null, - shuffleWriteMetrics = null))) + shuffleWriteMetrics = null)), + executorLogs = null, + schedulerDelay = 0L, + gettingResultTime = 0L) assert(AppStatusUtils.schedulerDelay(finishedTask) === 3L) } } From 495caafc800f33ae09b2d4ff0eb8ce33e31d7cc9 Mon Sep 17 00:00:00 2001 From: pgandhi Date: Wed, 20 Jun 2018 11:32:31 -0500 Subject: [PATCH 04/42] [SPARK-21809] : Rendering the ui compatible with current version using datatables --- .../org/apache/spark/ui/static/stagepage.js | 318 +++++++++++++++--- .../spark/ui/static/stagespage-template.html | 6 + .../spark/status/api/v1/StagesResource.scala | 51 ++- .../org/apache/spark/ui/jobs/StagePage.scala | 3 - 4 files changed, 328 insertions(+), 50 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js index 05f6024fff6dd..da014066ba471 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js @@ -68,7 +68,16 @@ function stageEndPoint(appId) { } function sortNumber(a,b) { - return a - b; + return a - b; +} + +function sortRecords(a,b) { + var aSplit = a.split("/"); + var bSplit = b.split("/"); + if (aSplit[0] == bSplit[0]) { + return aSplit[1] - bSplit[1]; + } + return aSplit[0] - bSplit[0]; } function quantile(array, percentile) { @@ -109,7 +118,6 @@ $(document).ready(function () { $.getJSON(endPoint, function(response, status, jqXHR) { // prepare data for tasks table - console.log("hereeeeeeeeee 1 "+JSON.stringify(response[0].tasks[0])); var indices = Object.keys(response[0].tasks); var task_table = []; indices.forEach(function (ix) { @@ -132,15 +140,38 @@ $(document).ready(function () { var resultSerializationTimeSummary = []; var gettingResultTimeSummary = []; var peakExecutionMemorySummary = []; + var inputSizeRecordsSummary = []; + var outputSizeRecordsSummary = []; + var shuffleWriteSizeRecordsSummary = []; + var shuffleSpillMemorySummary = []; + var shuffleSpillDiskSummary = []; + console.log("hereeeeeeeeee 1 "+task_table.length); task_table.forEach(function (x){ - durationSummary.push(x.taskMetrics.executorRunTime); + if ("taskMetrics" in x) { + durationSummary.push(x.taskMetrics.executorRunTime); + taskDeserializationSummary.push(x.taskMetrics.executorDeserializeTime); + resultSerializationTimeSummary.push(x.taskMetrics.resultSerializationTime); + gcTimeSummary.push(x.taskMetrics.jvmGcTime); + peakExecutionMemorySummary.push(x.taskMetrics.peakExecutionMemory); + if (x.taskMetrics.inputMetrics.bytesRead > 0) { + inputSizeRecordsSummary.push(x.taskMetrics.inputMetrics.bytesRead + "/" + x.taskMetrics.inputMetrics.recordsRead); + } + if (x.taskMetrics.outputMetrics.bytesWritten > 0) { + outputSizeRecordsSummary.push(x.taskMetrics.outputMetrics.bytesWritten + "/" + x.taskMetrics.outputMetrics.recordsWritten); + } + if (x.taskMetrics.shuffleWriteMetrics.bytesWritten > 0) { + shuffleWriteSizeRecordsSummary.push(x.taskMetrics.shuffleWriteMetrics.bytesWritten + "/" + x.taskMetrics.shuffleWriteMetrics.recordsWritten); + } + if (x.taskMetrics.memoryBytesSpilled > 0) { + shuffleSpillMemorySummary.push(x.taskMetrics.memoryBytesSpilled); + } + if (x.taskMetrics.diskBytesSpilled > 0) { + shuffleSpillDiskSummary.push(x.taskMetrics.diskBytesSpilled); + } + } schedulerDelaySummary.push(x.schedulerDelay); - taskDeserializationSummary.push(x.taskMetrics.executorDeserializeTime); - gcTimeSummary.push(x.taskMetrics.jvmGcTime); - resultSerializationTimeSummary.push(x.taskMetrics.resultSerializationTime); gettingResultTimeSummary.push(x.gettingResultTime); - peakExecutionMemorySummary.push(x.taskMetrics.peakExecutionMemory); }); var task_metrics_table = []; @@ -154,18 +185,44 @@ $(document).ready(function () { task_metrics_table_all.push(resultSerializationTimeSummary); task_metrics_table_all.push(gettingResultTimeSummary); task_metrics_table_all.push(peakExecutionMemorySummary); + if (inputSizeRecordsSummary.length > 0) { + task_metrics_table_all.push(inputSizeRecordsSummary); + task_metrics_table_col.push("Input Size / Records"); + } + if (outputSizeRecordsSummary.length > 0) { + task_metrics_table_all.push(outputSizeRecordsSummary); + task_metrics_table_col.push("Output Size / Records"); + } + if (shuffleWriteSizeRecordsSummary.length > 0) { + task_metrics_table_all.push(shuffleWriteSizeRecordsSummary); + task_metrics_table_col.push("Shuffle Write Size / Records"); + } + if (shuffleSpillMemorySummary.length > 0) { + task_metrics_table_all.push(shuffleSpillMemorySummary); + task_metrics_table_col.push("Shuffle spill (memory)"); + } + if (shuffleSpillDiskSummary.length > 0) { + task_metrics_table_all.push(shuffleSpillDiskSummary); + task_metrics_table_col.push("Shuffle spill (disk)"); + } for(i = 0; i < task_metrics_table_col.length; i++){ - var task_sort_table = (task_metrics_table_all[i]).sort(sortNumber); - var row = { - "metric": task_metrics_table_col[i], - "p0": quantile(task_sort_table, 0), - "p25": quantile(task_sort_table, 25), - "p50": quantile(task_sort_table, 50), - "p75": quantile(task_sort_table, 75), - "p100": quantile(task_sort_table, 100) - }; - task_metrics_table.push(row); + var task_sort_table; + if (task_metrics_table_col[i] == 'Input Size / Records' || task_metrics_table_col[i] == 'Output Size / Records' + || task_metrics_table_col[i] == 'Shuffle Write Size / Records') { + task_sort_table = (task_metrics_table_all[i]).sort(sortRecords); + } else { + task_sort_table = (task_metrics_table_all[i]).sort(sortNumber); + } + var row = { + "metric": task_metrics_table_col[i], + "p0": quantile(task_sort_table, 0), + "p25": quantile(task_sort_table, 25), + "p50": quantile(task_sort_table, 50), + "p75": quantile(task_sort_table, 75), + "p100": quantile(task_sort_table, 100) + }; + task_metrics_table.push(row); } // prepare data for accumulatorUpdates @@ -206,27 +263,67 @@ $(document).ready(function () { {data : 'metric'}, { data: function (row, type) { - return row.metric != 'Peak Execution Memory' ? (formatDuration(row.p0)) : formatBytes(row.p0, type); + if (row.metric == 'Input Size / Records' || row.metric == 'Output Size / Records' + || row.metric == 'Shuffle Write Size / Records') { + var strarray = row.p0.split("/"); + var str = formatBytes(strarray[0], type) + " / " + strarray[1]; + return str; + } else { + return (row.metric == 'Peak Execution Memory' || row.metric == 'Shuffle spill (memory)' + || row.metric == 'Shuffle spill (disk)') ? formatBytes(row.p0, type) : (formatDuration(row.p0)); + } } }, { data: function (row, type) { - return row.metric != 'Peak Execution Memory' ? (formatDuration(row.p25)) : formatBytes(row.p25, type); + if (row.metric == 'Input Size / Records' || row.metric == 'Output Size / Records' + || row.metric == 'Shuffle Write Size / Records') { + var strarray = row.p25.split("/"); + var str = formatBytes(strarray[0], type) + " / " + strarray[1]; + return str; + } else { + return (row.metric == 'Peak Execution Memory' || row.metric == 'Shuffle spill (memory)' + || row.metric == 'Shuffle spill (disk)') ? formatBytes(row.p25, type) : (formatDuration(row.p25)); + } } }, { data: function (row, type) { - return row.metric != 'Peak Execution Memory' ? (formatDuration(row.p50)) : formatBytes(row.p50, type); + if (row.metric == 'Input Size / Records' || row.metric == 'Output Size / Records' + || row.metric == 'Shuffle Write Size / Records') { + var strarray = row.p50.split("/"); + var str = formatBytes(strarray[0], type) + " / " + strarray[1]; + return str; + } else { + return (row.metric == 'Peak Execution Memory' || row.metric == 'Shuffle spill (memory)' + || row.metric == 'Shuffle spill (disk)') ? formatBytes(row.p50, type) : (formatDuration(row.p50)); + } } }, { data: function (row, type) { - return row.metric != 'Peak Execution Memory' ? (formatDuration(row.p75)) : formatBytes(row.p75, type); + if (row.metric == 'Input Size / Records' || row.metric == 'Output Size / Records' + || row.metric == 'Shuffle Write Size / Records') { + var strarray = row.p75.split("/"); + var str = formatBytes(strarray[0], type) + " / " + strarray[1]; + return str; + } else { + return (row.metric == 'Peak Execution Memory' || row.metric == 'Shuffle spill (memory)' + || row.metric == 'Shuffle spill (disk)') ? formatBytes(row.p75, type) : (formatDuration(row.p75)); + } } }, { data: function (row, type) { - return row.metric != 'Peak Execution Memory' ? (formatDuration(row.p100)) : formatBytes(row.p100, type); + if (row.metric == 'Input Size / Records' || row.metric == 'Output Size / Records' + || row.metric == 'Shuffle Write Size / Records') { + var strarray = row.p100.split("/"); + var str = formatBytes(strarray[0], type) + " / " + strarray[1]; + return str; + } else { + return (row.metric == 'Peak Execution Memory' || row.metric == 'Shuffle spill (memory)' + || row.metric == 'Shuffle spill (disk)') ? formatBytes(row.p100, type) : (formatDuration(row.p100)); + } } } ], @@ -268,32 +365,32 @@ $(document).ready(function () { {data : "isBlacklistedForStage"}, { data : function (row, type) { - return row.inputRecords != 0 ? formatBytes(row.inputBytes/row.inputRecords) : 0; + return row.inputRecords != 0 ? formatBytes(row.inputBytes, type) + " / " + row.inputRecords : 0; } }, { data : function (row, type) { - return row.outputRecords != 0 ? formatBytes(row.outputBytes/row.outputRecords) : 0; + return row.outputRecords != 0 ? formatBytes(row.outputBytes, type) + " / " + row.outputRecords : 0; } }, { data : function (row, type) { - return row.shuffleReadRecords != 0 ? formatBytes(row.shuffleRead/row.shuffleReadRecords) : 0; + return row.shuffleReadRecords != 0 ? formatBytes(row.shuffleRead, type) + " / " + row.shuffleReadRecords : 0; } }, { data : function (row, type) { - return row.shuffleWriteRecords != 0 ? formatBytes(row.shuffleWrite/row.shuffleWriteRecords) : 0; + return row.shuffleWriteRecords != 0 ? formatBytes(row.shuffleWrite, type) + " / " + row.shuffleWriteRecords : 0; } }, { data : function (row, type) { - return typeof row.memoryBytesSpilled != 'undefined' ? formatBytes(row.memoryBytesSpilled) : ""; + return typeof row.memoryBytesSpilled != 'undefined' ? formatBytes(row.memoryBytesSpilled, type) : ""; } }, { data : function (row, type) { - return typeof row.diskBytesSpilled != 'undefined' ? formatBytes(row.diskBytesSpilled) : ""; + return typeof row.diskBytesSpilled != 'undefined' ? formatBytes(row.diskBytesSpilled, type) : ""; } } ], @@ -321,6 +418,8 @@ $(document).ready(function () { var taskTable = "#active-tasks-table"; var task_conf = { "data": task_table, + //"serverSide": true, + //"ajax": stageEndPoint(appId) + "/taskTable", "columns": [ {data: function (row, type) { return type !== 'display' ? (isNaN(row.index) ? 0 : row.index ) : row.index; @@ -339,12 +438,20 @@ $(document).ready(function () { {data : "launchTime", render: formatDate}, { data : function (row, type) { - return type === 'display' ? formatDuration(row.taskMetrics.executorRunTime) : row.taskMetrics.executorRunTime; + if ("taskMetrics" in row) { + return type === 'display' ? formatDuration(row.taskMetrics.executorRunTime) : row.taskMetrics.executorRunTime; + } else { + return "N/A"; + } } }, { data : function (row, type) { - return type === 'display' ? formatDuration(row.taskMetrics.jvmGcTime) : row.taskMetrics.jvmGcTime; + if ("taskMetrics" in row) { + return type === 'display' ? formatDuration(row.taskMetrics.jvmGcTime) : row.taskMetrics.jvmGcTime; + } else { + return "N/A"; + } } }, { @@ -354,22 +461,38 @@ $(document).ready(function () { }, { data : function (row, type) { - return type === 'display' ? formatDuration(row.taskMetrics.executorDeserializeTime) : row.taskMetrics.executorDeserializeTime; + if ("taskMetrics" in row) { + return type === 'display' ? formatDuration(row.taskMetrics.executorDeserializeTime) : row.taskMetrics.executorDeserializeTime; + } else { + return "N/A"; + } } }, { data : function (row, type) { - return type === 'display' ? formatDuration(row.taskMetrics.shuffleReadMetrics.fetchWaitTime) : row.taskMetrics.shuffleReadMetrics.fetchWaitTime; + if ("taskMetrics" in row) { + return type === 'display' ? formatDuration(row.taskMetrics.shuffleReadMetrics.fetchWaitTime) : row.taskMetrics.shuffleReadMetrics.fetchWaitTime; + } else { + return "N/A"; + } } }, { data : function (row, type) { - return type === 'display' ? formatBytes(row.taskMetrics.shuffleReadMetrics.remoteBytesRead, type) : row.taskMetrics.shuffleReadMetrics.remoteBytesRead; + if ("taskMetrics" in row) { + return type === 'display' ? formatBytes(row.taskMetrics.shuffleReadMetrics.remoteBytesRead, type) : row.taskMetrics.shuffleReadMetrics.remoteBytesRead; + } else { + return "N/A"; + } } }, { data : function (row, type) { - return type === 'display' ? formatDuration(row.taskMetrics.resultSerializationTime) : row.taskMetrics.resultSerializationTime; + if ("taskMetrics" in row) { + return type === 'display' ? formatDuration(row.taskMetrics.resultSerializationTime) : row.taskMetrics.resultSerializationTime; + } else { + return "N/A"; + } } }, { @@ -379,7 +502,11 @@ $(document).ready(function () { }, { data : function (row, type) { - return type === 'display' ? formatBytes(row.taskMetrics.peakExecutionMemory, type) : row.taskMetrics.peakExecutionMemory; + if ("taskMetrics" in row) { + return type === 'display' ? formatBytes(row.taskMetrics.peakExecutionMemory, type) : row.taskMetrics.peakExecutionMemory; + } else { + return "N/A"; + } } }, { @@ -391,6 +518,96 @@ $(document).ready(function () { } } }, + { + data : function (row, type) { + if ("taskMetrics" in row) { + if (row.taskMetrics.inputMetrics.bytesRead > 0) { + if (type === 'display') { + return formatBytes(row.taskMetrics.inputMetrics.bytesRead, type) + " / " + row.taskMetrics.inputMetrics.recordsRead; + } else { + return row.taskMetrics.inputMetrics.bytesRead + " / " + row.taskMetrics.inputMetrics.recordsRead; + } + } else { + return ""; + } + } else { + return ""; + } + } + }, + { + data : function (row, type) { + if ("taskMetrics" in row) { + if (row.taskMetrics.outputMetrics.bytesWritten > 0) { + if (type === 'display') { + return formatBytes(row.taskMetrics.outputMetrics.bytesWritten, type) + " / " + row.taskMetrics.outputMetrics.recordsWritten; + } else { + return row.taskMetrics.outputMetrics.bytesWritten + " / " + row.taskMetrics.outputMetrics.recordsWritten; + } + } else { + return ""; + } + } else { + return ""; + } + } + }, + { + data : function (row, type) { + if ("taskMetrics" in row) { + if (row.taskMetrics.shuffleWriteMetrics.writeTime > 0) { + return type === 'display' ? formatDuration(row.taskMetrics.shuffleWriteMetrics.writeTime) : row.taskMetrics.shuffleWriteMetrics.writeTime; + } else { + return ""; + } + } else { + return ""; + } + } + }, + { + data : function (row, type) { + if ("taskMetrics" in row) { + if (row.taskMetrics.shuffleWriteMetrics.bytesWritten > 0) { + if (type === 'display') { + return formatBytes(row.taskMetrics.shuffleWriteMetrics.bytesWritten, type) + " / " + row.taskMetrics.shuffleWriteMetrics.recordsWritten; + } else { + return row.taskMetrics.shuffleWriteMetrics.bytesWritten + " / " + row.taskMetrics.shuffleWriteMetrics.recordsWritten; + } + } else { + return ""; + } + } else { + return ""; + } + } + }, + { + data : function (row, type) { + if ("taskMetrics" in row) { + if (row.taskMetrics.memoryBytesSpilled > 0) { + return type === 'display' ? formatBytes(row.taskMetrics.memoryBytesSpilled, type) : row.taskMetrics.memoryBytesSpilled; + } else { + return ""; + } + } else { + return ""; + } + } + }, + { + data : function (row, type) { + if ("taskMetrics" in row) { + if (row.taskMetrics.diskBytesSpilled > 0) { + return type === 'display' ? formatBytes(row.taskMetrics.diskBytesSpilled, type) : row.taskMetrics.diskBytesSpilled; + } else { + return ""; + } + } else { + return ""; + } + } + }, { data : function (row, type) { var msg = row.errorMessage; @@ -413,7 +630,13 @@ $(document).ready(function () { { "visible": false, "targets": 14 }, { "visible": false, "targets": 15 }, { "visible": false, "targets": 16 }, - { "visible": false, "targets": 17 } + { "visible": false, "targets": 17 }, + { "visible": false, "targets": 18 }, + { "visible": false, "targets": 19 }, + { "visible": false, "targets": 20 }, + { "visible": false, "targets": 21 }, + { "visible": false, "targets": 22 }, + { "visible": false, "targets": 23 } ], "order": [[0, "asc"]] }; @@ -453,8 +676,8 @@ $(document).ready(function () { }); // title number and toggle list - $("#summaryMetricsTitle").html("Summary Metrics for " + "" + task_table.length + " Completed Tasks" + ""); - $("#tasksTitle").html("Task (" + task_table.length + ")"); + $("#summaryMetricsTitle").html("Summary Metrics for " + "" + response[0].numCompleteTasks + " Completed Tasks" + ""); + $("#tasksTitle").html("Task (" + response[0].numCompleteTasks + ")"); // hide or show the accumulate update table if (accumulator_table.length == 0) { @@ -463,6 +686,25 @@ $(document).ready(function () { taskTableSelector.column(17).visible(true); $("#accumulator-update-table").show(); } + + if (inputSizeRecordsSummary.length > 0) { + taskTableSelector.column(18).visible(true); + } + if (outputSizeRecordsSummary.length > 0) { + taskTableSelector.column(19).visible(true); + } + if (shuffleWriteSizeRecordsSummary.length > 0) { + taskTableSelector.column(20).visible(true); + } + if (shuffleWriteSizeRecordsSummary.length > 0) { + taskTableSelector.column(21).visible(true); + } + if (shuffleSpillMemorySummary.length > 0) { + taskTableSelector.column(22).visible(true); + } + if (shuffleSpillDiskSummary.length > 0) { + taskTableSelector.column(23).visible(true); + } }); }); }); diff --git a/core/src/main/resources/org/apache/spark/ui/static/stagespage-template.html b/core/src/main/resources/org/apache/spark/ui/static/stagespage-template.html index 1df5765744d6b..2c7c42419c85b 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/stagespage-template.html +++ b/core/src/main/resources/org/apache/spark/ui/static/stagespage-template.html @@ -105,6 +105,12 @@

Getting Result Time Peak Execution Memory Accumulators + Input Size / Records + Output Size / Records + Write Time + Shuffle Write Size / Records + Shuffle Spill (Memory) + Shuffle Spill (Disk) Errors diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala index 288188b7d84eb..9c8097a54e000 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala @@ -44,21 +44,19 @@ private[v1] class StagesResource extends BaseAppResource { var ret = ui.store.stageData(stageId, details = details) if (ret.nonEmpty) { for (i <- 0 to (ret.length - 1)) { - val executorIdArray = ret(i).executorSummary.get.keys.toArray + var executorIdArray = ret(i).executorSummary.get.keys.toArray for (execId <- executorIdArray) { - val executorLogs = ui.store.executorSummary(execId).executorLogs - val hostPort = ui.store.executorSummary(execId).hostPort - val taskDataArray = ret(i).tasks.get.keys.toArray - val executorStageSummaryArray = ret(i).executorSummary.get.keys.toArray + var executorLogs = ui.store.executorSummary(execId).executorLogs + var hostPort = ui.store.executorSummary(execId).hostPort + var taskDataArray = ret(i).tasks.get.keys.toArray + var executorStageSummaryArray = ret(i).executorSummary.get.keys.toArray + ret(i).executorSummary.get.get(execId).get.executorLogs = executorLogs + ret(i).executorSummary.get.get(execId).get.hostPort = hostPort for (taskData <- taskDataArray) { ret(i).tasks.get.get(taskData).get.executorLogs = executorLogs ret(i).tasks.get.get(taskData).get.schedulerDelay = AppStatusUtils.schedulerDelay(ret(i).tasks.get.get(taskData).get) ret(i).tasks.get.get(taskData).get.gettingResultTime = AppStatusUtils.gettingResultTime(ret(i).tasks.get.get(taskData).get) } - for (executorStageSummary <- executorStageSummaryArray) { - ret(i).executorSummary.get.get(executorStageSummary).get.executorLogs = executorLogs - ret(i).executorSummary.get.get(executorStageSummary).get.hostPort = hostPort - } } } ret @@ -121,4 +119,39 @@ private[v1] class StagesResource extends BaseAppResource { withUI(_.store.taskList(stageId, stageAttemptId, offset, length, sortBy)) } + /*@GET + @Path("{stageId: \\d+}/taskTable") + def taskTable( + @PathParam("stageId") stageId: Int, + @QueryParam("details") @DefaultValue("true") details: Boolean): Map[String, TaskData] = { + withUI { ui => + var ret = ui.store.stageData(stageId, details = details) + if (ret.nonEmpty) { + for (i <- 0 to (ret.length - 1)) { + var executorIdArray = ret(i).executorSummary.get.keys.toArray + for (execId <- executorIdArray) { + var executorLogs = ui.store.executorSummary(execId).executorLogs + var hostPort = ui.store.executorSummary(execId).hostPort + var taskDataArray = ret(i).tasks.get.keys.toArray + var executorStageSummaryArray = ret(i).executorSummary.get.keys.toArray + ret(i).executorSummary.get.get(execId).get.executorLogs = executorLogs + ret(i).executorSummary.get.get(execId).get.hostPort = hostPort + for (taskData <- taskDataArray) { + ret(i).tasks.get.get(taskData).get.executorLogs = executorLogs + ret(i).tasks.get.get(taskData).get.schedulerDelay = AppStatusUtils.schedulerDelay(ret(i).tasks.get.get(taskData).get) + ret(i).tasks.get.get(taskData).get.gettingResultTime = AppStatusUtils.gettingResultTime(ret(i).tasks.get.get(taskData).get) + } + } + } + var ret1:Map[String, TaskData] = Map() + ret(0).tasks.get.keys.foreach({ i => + ret1 += (i.toString -> ret(0).tasks.get(i)) + }) + ret1 + } else { + throw new NotFoundException(s"unknown stage: $stageId") + } + } + }*/ + } 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 1c73698127eff..c406b0bfddf06 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 @@ -231,9 +231,6 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We } - val metricsSummary = store.taskSummary(stageData.stageId, stageData.attemptId, - Array(0, 0.25, 0.5, 0.75, 1.0)) - val content = summary ++ dagViz ++
++ From c1d41d836f4d577e1188ca42e20abf9940f818ab Mon Sep 17 00:00:00 2001 From: pgandhi Date: Fri, 22 Jun 2018 17:10:15 -0500 Subject: [PATCH 05/42] Working on Adding Server Side Sort Functionality --- .../org/apache/spark/ui/static/stagepage.js | 93 ++++++++++++------- .../spark/status/api/v1/StagesResource.scala | 36 +++++-- .../status/api/v1/TaskDataComparator.scala | 12 +++ 3 files changed, 102 insertions(+), 39 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/status/api/v1/TaskDataComparator.scala diff --git a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js index da014066ba471..b15026fe8f6d4 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js @@ -417,25 +417,36 @@ $(document).ready(function () { // building tasks table var taskTable = "#active-tasks-table"; var task_conf = { - "data": task_table, - //"serverSide": true, - //"ajax": stageEndPoint(appId) + "/taskTable", + "serverSide": true, + "paging": false, + "info": true, + "processing": true, + "ajax": { + "url": stageEndPoint(appId) + "/taskTable", + "dataSrc": function ( jsons ) { + var jsonStr = JSON.stringify(jsons); + var marrr = JSON.parse(jsonStr); + return marrr.aaData; + } + }, "columns": [ {data: function (row, type) { return type !== 'display' ? (isNaN(row.index) ? 0 : row.index ) : row.index; - } + }, + name: "Index" }, - {data : "taskId"}, - {data : "attempt"}, - {data : "status"}, - {data : "taskLocality"}, + {data : "taskId", name: "ID"}, + {data : "attempt", name: "Attempt"}, + {data : "status", name: "Status"}, + {data : "taskLocality", name: "Locality Level"}, { data : function (row, type) { return row.executorId + ' / ' + row.host; - } + }, + name: "Executor ID" }, - {data : "executorLogs", render: formatLogsCells}, - {data : "launchTime", render: formatDate}, + {data : "executorLogs", name: "Logs", render: formatLogsCells}, + {data : "launchTime", name: "Launch Time", render: formatDate}, { data : function (row, type) { if ("taskMetrics" in row) { @@ -443,7 +454,8 @@ $(document).ready(function () { } else { return "N/A"; } - } + }, + name: "Duration" }, { data : function (row, type) { @@ -452,12 +464,14 @@ $(document).ready(function () { } else { return "N/A"; } - } + }, + name: "GC Time" }, { data : function (row, type) { return type === 'display' ? formatDuration(row.schedulerDelay) : row.schedulerDelay; - } + }, + name: "Scheduler Delay" }, { data : function (row, type) { @@ -466,16 +480,22 @@ $(document).ready(function () { } else { return "N/A"; } - } + }, + name: "Task Deserialization Time" }, { data : function (row, type) { if ("taskMetrics" in row) { - return type === 'display' ? formatDuration(row.taskMetrics.shuffleReadMetrics.fetchWaitTime) : row.taskMetrics.shuffleReadMetrics.fetchWaitTime; + if (row.taskMetrics.shuffleReadMetrics !== 'undefined') { + return type === 'display' ? formatDuration(row.taskMetrics.shuffleReadMetrics.fetchWaitTime) : row.taskMetrics.shuffleReadMetrics.fetchWaitTime; + } else { + return ""; + } } else { - return "N/A"; + return ""; } - } + }, + name: "Shuffle Read Blocked Time" }, { data : function (row, type) { @@ -484,7 +504,8 @@ $(document).ready(function () { } else { return "N/A"; } - } + }, + name: "Shuffle Remote Reads" }, { data : function (row, type) { @@ -493,12 +514,14 @@ $(document).ready(function () { } else { return "N/A"; } - } + }, + name: "Result Serialization Time" }, { data : function (row, type) { return type === 'display' ? formatDuration(row.gettingResultTime) : row.gettingResultTime; - } + }, + name: "Getting Result Time" }, { data : function (row, type) { @@ -507,7 +530,8 @@ $(document).ready(function () { } else { return "N/A"; } - } + }, + name: "Peak Execution Memory" }, { data : function (row, type) { @@ -516,7 +540,8 @@ $(document).ready(function () { } else { return ""; } - } + }, + name: "Accumulators" }, { data : function (row, type) { @@ -533,7 +558,8 @@ $(document).ready(function () { } else { return ""; } - } + }, + name: "Input Size / Records" }, { data : function (row, type) { @@ -550,7 +576,8 @@ $(document).ready(function () { } else { return ""; } - } + }, + name: "Output Size / Records" }, { data : function (row, type) { @@ -563,7 +590,8 @@ $(document).ready(function () { } else { return ""; } - } + }, + name: "Write Time" }, { data : function (row, type) { @@ -580,7 +608,8 @@ $(document).ready(function () { } else { return ""; } - } + }, + name: "Shuffle Write Size / Records" }, { data : function (row, type) { @@ -593,7 +622,8 @@ $(document).ready(function () { } else { return ""; } - } + }, + name: "Shuffle Spill (Memory)" }, { data : function (row, type) { @@ -606,7 +636,8 @@ $(document).ready(function () { } else { return ""; } - } + }, + name: "Shuffle Spill (Disk)" }, { data : function (row, type) { @@ -619,7 +650,8 @@ $(document).ready(function () { var form_msg = "
" + row.errorMessage + "
"; return form_head + form + form_msg; } - } + }, + name: "Errors" } ], "columnDefs": [ @@ -638,7 +670,6 @@ $(document).ready(function () { { "visible": false, "targets": 22 }, { "visible": false, "targets": 23 } ], - "order": [[0, "asc"]] }; var taskTableSelector = $(taskTable).DataTable(task_conf); diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala index 9c8097a54e000..b24890c535839 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala @@ -16,9 +16,10 @@ */ package org.apache.spark.status.api.v1 -import java.util.{List => JList} +import java.util +import java.util.{Collections, Comparator, List => JList} import javax.ws.rs._ -import javax.ws.rs.core.MediaType +import javax.ws.rs.core.{Context, MediaType, UriInfo} import org.apache.spark.SparkException import org.apache.spark.scheduler.StageInfo @@ -119,13 +120,19 @@ private[v1] class StagesResource extends BaseAppResource { withUI(_.store.taskList(stageId, stageAttemptId, offset, length, sortBy)) } - /*@GET + @GET @Path("{stageId: \\d+}/taskTable") def taskTable( @PathParam("stageId") stageId: Int, - @QueryParam("details") @DefaultValue("true") details: Boolean): Map[String, TaskData] = { + @QueryParam("details") @DefaultValue("true") details: Boolean, @Context uriInfo: UriInfo): util.HashMap[String, util.ArrayList[TaskData]] = { withUI { ui => var ret = ui.store.stageData(stageId, details = details) + var abc = uriInfo.getQueryParameters(true) + var iter = abc.keySet().iterator() + while(iter.hasNext) { + var ac = iter.next() + System.err.println("hereeeeeeeeeeee 1 " + ac+" : "+abc.get(ac)) + } if (ret.nonEmpty) { for (i <- 0 to (ret.length - 1)) { var executorIdArray = ret(i).executorSummary.get.keys.toArray @@ -143,15 +150,28 @@ private[v1] class StagesResource extends BaseAppResource { } } } - var ret1:Map[String, TaskData] = Map() + val ret4 = new util.ArrayList[TaskData]() ret(0).tasks.get.keys.foreach({ i => - ret1 += (i.toString -> ret(0).tasks.get(i)) + ret4.add(ret(0).tasks.get(i)); + }) + + Collections.sort(ret4, new Comparator[TaskData] { + def compare(t1: TaskData, t2: TaskData): Int = { + var str = "index" + var c = t1.getClass + val result = c.getField(str).get(t1) + if (t1.index == t2.index) return 0 + if (t1.index < t2.index) -1 + else 1 + } }) - ret1 + val ret5 = new util.HashMap[String, util.ArrayList[TaskData]]() + ret5.put("aaData", ret4) + ret5 } else { throw new NotFoundException(s"unknown stage: $stageId") } } - }*/ + } } diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/TaskDataComparator.scala b/core/src/main/scala/org/apache/spark/status/api/v1/TaskDataComparator.scala new file mode 100644 index 0000000000000..d35e96d22daf3 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/TaskDataComparator.scala @@ -0,0 +1,12 @@ +package org.apache.spark.status.api.v1 + +import java.util.Comparator + +class TaskDataComparator(var index: Int) extends Comparator[TaskData] { + override def compare(o1: TaskData, o2: TaskData): Int = { + + if (o1.index == o2.index) return 0 + if (o1.index < o2.index) -1 + else 1 + } +} From 567d22b07a7c434141b2d8e8bcc183f87b0d8d52 Mon Sep 17 00:00:00 2001 From: pgandhi Date: Mon, 25 Jun 2018 16:39:47 -0500 Subject: [PATCH 06/42] [SPARK-21809] : Implement Server Side Sorting Functionality --- .../org/apache/spark/ui/static/stagepage.js | 32 ++++----- .../spark/ui/static/stagespage-template.html | 3 +- .../apache/spark/status/AppStatusStore.scala | 20 ++++++ .../spark/status/api/v1/StagesResource.scala | 66 ++++++++----------- .../org/apache/spark/ui/jobs/StagePage.scala | 2 +- 5 files changed, 66 insertions(+), 57 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js index b15026fe8f6d4..fe8777910f959 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js @@ -439,12 +439,8 @@ $(document).ready(function () { {data : "attempt", name: "Attempt"}, {data : "status", name: "Status"}, {data : "taskLocality", name: "Locality Level"}, - { - data : function (row, type) { - return row.executorId + ' / ' + row.host; - }, - name: "Executor ID" - }, + {data : "executorId", name: "Executor ID"}, + {data : "host", name: "Host"}, {data : "executorLogs", name: "Logs", render: formatLogsCells}, {data : "launchTime", name: "Launch Time", render: formatDate}, { @@ -583,7 +579,7 @@ $(document).ready(function () { data : function (row, type) { if ("taskMetrics" in row) { if (row.taskMetrics.shuffleWriteMetrics.writeTime > 0) { - return type === 'display' ? formatDuration(row.taskMetrics.shuffleWriteMetrics.writeTime) : row.taskMetrics.shuffleWriteMetrics.writeTime; + return type === 'display' ? formatDuration(parseInt(row.taskMetrics.shuffleWriteMetrics.writeTime) / 1000000) : row.taskMetrics.shuffleWriteMetrics.writeTime; } else { return ""; } @@ -655,7 +651,6 @@ $(document).ready(function () { } ], "columnDefs": [ - { "visible": false, "targets": 10 }, { "visible": false, "targets": 11 }, { "visible": false, "targets": 12 }, { "visible": false, "targets": 13 }, @@ -668,12 +663,13 @@ $(document).ready(function () { { "visible": false, "targets": 20 }, { "visible": false, "targets": 21 }, { "visible": false, "targets": 22 }, - { "visible": false, "targets": 23 } + { "visible": false, "targets": 23 }, + { "visible": false, "targets": 24 } ], }; var taskTableSelector = $(taskTable).DataTable(task_conf); - var optionalColumns = [10, 11, 12, 13, 14, 15, 16]; + var optionalColumns = [11, 12, 13, 14, 15, 16, 17]; var allChecked = true; for(k = 0; k < optionalColumns.length; k++) { if (taskTableSelector.column(optionalColumns[k]).visible()) { @@ -691,7 +687,7 @@ $(document).ready(function () { // Get the column var para = $(this).attr('data-column'); if(para == "0"){ - var column = taskTableSelector.column([10, 11, 12, 13, 14, 15, 16]); + var column = taskTableSelector.column([11, 12, 13, 14, 15, 16, 17]); if($(this).is(":checked")){ $(".toggle-vis").prop('checked', true); column.visible(true); @@ -714,28 +710,28 @@ $(document).ready(function () { if (accumulator_table.length == 0) { $("#accumulator-update-table").hide(); } else { - taskTableSelector.column(17).visible(true); + taskTableSelector.column(18).visible(true); $("#accumulator-update-table").show(); } if (inputSizeRecordsSummary.length > 0) { - taskTableSelector.column(18).visible(true); - } - if (outputSizeRecordsSummary.length > 0) { taskTableSelector.column(19).visible(true); } - if (shuffleWriteSizeRecordsSummary.length > 0) { + if (outputSizeRecordsSummary.length > 0) { taskTableSelector.column(20).visible(true); } if (shuffleWriteSizeRecordsSummary.length > 0) { taskTableSelector.column(21).visible(true); } - if (shuffleSpillMemorySummary.length > 0) { + if (shuffleWriteSizeRecordsSummary.length > 0) { taskTableSelector.column(22).visible(true); } - if (shuffleSpillDiskSummary.length > 0) { + if (shuffleSpillMemorySummary.length > 0) { taskTableSelector.column(23).visible(true); } + if (shuffleSpillDiskSummary.length > 0) { + taskTableSelector.column(24).visible(true); + } }); }); }); diff --git a/core/src/main/resources/org/apache/spark/ui/static/stagespage-template.html b/core/src/main/resources/org/apache/spark/ui/static/stagespage-template.html index 2c7c42419c85b..8f2f7a504bbd0 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/stagespage-template.html +++ b/core/src/main/resources/org/apache/spark/ui/static/stagespage-template.html @@ -92,7 +92,8 @@

Attempt Status Locality level - Executor ID / Host + Executor ID + Host Logs Launch Time Duration diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala index 688f25a9fdea1..c2328350b6184 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala @@ -391,6 +391,26 @@ private[spark] class AppStatusStore( ordered.skip(offset).max(length).asScala.map(_.toApi).toSeq } + def fullTaskList( + stageId: Int, + stageAttemptId: Int, + sortBy: Option[String], + ascending: Boolean): Seq[v1.TaskData] = { + val stageKey = Array(stageId, stageAttemptId) + val base = store.view(classOf[TaskDataWrapper]) + val indexed = sortBy match { + case Some(index) => + base.index(index).parent(stageKey) + + case _ => + // Sort by ID, which is the "stage" index. + base.index("stage").first(stageKey).last(stageKey) + } + + val ordered = if (ascending) indexed else indexed.reverse() + ordered.skip(0).asScala.map(_.toApi).toSeq + } + def executorSummary(stageId: Int, attemptId: Int): Map[String, v1.ExecutorStageSummary] = { val stageKey = Array(stageId, attemptId) store.view(classOf[ExecutorStageSummaryWrapper]).index("stage").first(stageKey).last(stageKey) diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala index b24890c535839..5362ad08bf349 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala @@ -27,6 +27,7 @@ import org.apache.spark.status.AppStatusUtils import org.apache.spark.status.api.v1.StageStatus._ import org.apache.spark.status.api.v1.TaskSorting._ import org.apache.spark.ui.SparkUI +import org.apache.spark.ui.jobs.ApiHelper._ @Produces(Array(MediaType.APPLICATION_JSON)) private[v1] class StagesResource extends BaseAppResource { @@ -124,49 +125,40 @@ private[v1] class StagesResource extends BaseAppResource { @Path("{stageId: \\d+}/taskTable") def taskTable( @PathParam("stageId") stageId: Int, - @QueryParam("details") @DefaultValue("true") details: Boolean, @Context uriInfo: UriInfo): util.HashMap[String, util.ArrayList[TaskData]] = { + @QueryParam("details") @DefaultValue("true") details: Boolean, @Context uriInfo: UriInfo): util.HashMap[String, Seq[TaskData]] = { withUI { ui => - var ret = ui.store.stageData(stageId, details = details) - var abc = uriInfo.getQueryParameters(true) - var iter = abc.keySet().iterator() + val abc = uriInfo.getQueryParameters(true) + val iter = abc.keySet().iterator() while(iter.hasNext) { var ac = iter.next() System.err.println("hereeeeeeeeeeee 1 " + ac+" : "+abc.get(ac)) } - if (ret.nonEmpty) { - for (i <- 0 to (ret.length - 1)) { - var executorIdArray = ret(i).executorSummary.get.keys.toArray - for (execId <- executorIdArray) { - var executorLogs = ui.store.executorSummary(execId).executorLogs - var hostPort = ui.store.executorSummary(execId).hostPort - var taskDataArray = ret(i).tasks.get.keys.toArray - var executorStageSummaryArray = ret(i).executorSummary.get.keys.toArray - ret(i).executorSummary.get.get(execId).get.executorLogs = executorLogs - ret(i).executorSummary.get.get(execId).get.hostPort = hostPort - for (taskData <- taskDataArray) { - ret(i).tasks.get.get(taskData).get.executorLogs = executorLogs - ret(i).tasks.get.get(taskData).get.schedulerDelay = AppStatusUtils.schedulerDelay(ret(i).tasks.get.get(taskData).get) - ret(i).tasks.get.get(taskData).get.gettingResultTime = AppStatusUtils.gettingResultTime(ret(i).tasks.get.get(taskData).get) - } - } - } - val ret4 = new util.ArrayList[TaskData]() - ret(0).tasks.get.keys.foreach({ i => - ret4.add(ret(0).tasks.get(i)); - }) + val queryParams = abc.keySet() + var columnToSort = 0 + if (queryParams.contains("order[0][column]")) { + columnToSort = abc.getFirst("order[0][column]").toInt + } + var columnNameToSort = abc.getFirst("columns["+columnToSort+"][name]") + if (columnNameToSort.equalsIgnoreCase("Logs")) { + columnNameToSort = "Index" + columnToSort = 0 + } + val isAscendingStr = abc.getFirst("order[0][dir]") + val _tasksToShow: Seq[TaskData] = ui.store.fullTaskList(stageId, 0, + indexName(columnNameToSort), isAscendingStr.equalsIgnoreCase("asc")) + if (_tasksToShow.nonEmpty) { - Collections.sort(ret4, new Comparator[TaskData] { - def compare(t1: TaskData, t2: TaskData): Int = { - var str = "index" - var c = t1.getClass - val result = c.getField(str).get(t1) - if (t1.index == t2.index) return 0 - if (t1.index < t2.index) -1 - else 1 - } - }) - val ret5 = new util.HashMap[String, util.ArrayList[TaskData]]() - ret5.put("aaData", ret4) + val iterator = _tasksToShow.iterator + while(iterator.hasNext) { + val t1: TaskData = iterator.next() + val execId = t1.executorId + val executorLogs = ui.store.executorSummary(execId).executorLogs + t1.executorLogs = executorLogs + t1.schedulerDelay = AppStatusUtils.schedulerDelay(t1) + t1.gettingResultTime = AppStatusUtils.gettingResultTime(t1) + } + val ret5 = new util.HashMap[String, Seq[TaskData]]() + ret5.put("aaData", _tasksToShow) ret5 } else { throw new NotFoundException(s"unknown stage: $stageId") 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 c406b0bfddf06..f3dcbee962586 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 @@ -709,7 +709,7 @@ private[ui] class TaskPagedTable( } } -private[ui] object ApiHelper { +private[spark] object ApiHelper { val HEADER_ID = "ID" val HEADER_TASK_INDEX = "Index" From 649e95b86c7dd9c09173e11bcadfe9b6855b44a4 Mon Sep 17 00:00:00 2001 From: pgandhi Date: Wed, 27 Jun 2018 13:14:18 -0500 Subject: [PATCH 07/42] [SPARK-21809] : Implementing server side pagination and searching functionality --- .../org/apache/spark/ui/static/stagepage.js | 11 ++- .../apache/spark/status/AppStatusStore.scala | 47 +++++++----- .../spark/status/api/v1/StagesResource.scala | 71 ++++++++++++++----- .../status/api/v1/TaskDataComparator.scala | 12 ---- 4 files changed, 93 insertions(+), 48 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/status/api/v1/TaskDataComparator.scala diff --git a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js index fe8777910f959..1d52520821562 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js @@ -418,11 +418,14 @@ $(document).ready(function () { var taskTable = "#active-tasks-table"; var task_conf = { "serverSide": true, - "paging": false, + "paging": true, "info": true, "processing": true, "ajax": { "url": stageEndPoint(appId) + "/taskTable", + "data": { + "numTasks": response[0].numTasks + }, "dataSrc": function ( jsons ) { var jsonStr = JSON.stringify(jsons); var marrr = JSON.parse(jsonStr); @@ -668,6 +671,12 @@ $(document).ready(function () { ], }; var taskTableSelector = $(taskTable).DataTable(task_conf); + $('#active-tasks-table_filter input').unbind(); + $('#active-tasks-table_filter input').bind('keyup', function(e) { + if(e.keyCode == 13) { + taskTableSelector.search( this.value ).draw(); + } + }); var optionalColumns = [11, 12, 13, 14, 15, 16, 17]; var allChecked = true; diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala index c2328350b6184..b20b212856ad9 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala @@ -391,24 +391,35 @@ private[spark] class AppStatusStore( ordered.skip(offset).max(length).asScala.map(_.toApi).toSeq } - def fullTaskList( - stageId: Int, - stageAttemptId: Int, - sortBy: Option[String], - ascending: Boolean): Seq[v1.TaskData] = { - val stageKey = Array(stageId, stageAttemptId) - val base = store.view(classOf[TaskDataWrapper]) - val indexed = sortBy match { - case Some(index) => - base.index(index).parent(stageKey) - - case _ => - // Sort by ID, which is the "stage" index. - base.index("stage").first(stageKey).last(stageKey) - } - - val ordered = if (ascending) indexed else indexed.reverse() - ordered.skip(0).asScala.map(_.toApi).toSeq + def filterTaskList( + taskDataList: Seq[v1.TaskData], + searchValue: String): Seq[v1.TaskData] = { + val defaultOptionString: String = "d" + val filteredTaskDataSequence: Seq[v1.TaskData] = taskDataList.filter(f => + (f.taskId.toString.contains(searchValue) || f.index.toString.contains(searchValue) + || f.attempt.toString.contains(searchValue) || f.launchTime.toString.contains(searchValue) + || f.resultFetchStart.getOrElse(defaultOptionString).toString.contains(searchValue) + || f.duration.getOrElse(defaultOptionString).toString.contains(searchValue) + || f.executorId.contains(searchValue) || f.host.contains(searchValue) + || f.status.contains(searchValue) || f.taskLocality.contains(searchValue) + || f.speculative.toString.contains(searchValue) + || f.errorMessage.getOrElse(defaultOptionString).contains(searchValue) + || f.taskMetrics.get.executorDeserializeTime.toString.contains(searchValue) + || f.taskMetrics.get.executorRunTime.toString.contains(searchValue) + || f.taskMetrics.get.jvmGcTime.toString.contains(searchValue) + || f.taskMetrics.get.resultSerializationTime.toString.contains(searchValue) + || f.taskMetrics.get.memoryBytesSpilled.toString.contains(searchValue) + || f.taskMetrics.get.diskBytesSpilled.toString.contains(searchValue) + || f.taskMetrics.get.peakExecutionMemory.toString.contains(searchValue) + || f.taskMetrics.get.inputMetrics.bytesRead.toString.contains(searchValue) + || f.taskMetrics.get.inputMetrics.recordsRead.toString.contains(searchValue) + || f.taskMetrics.get.outputMetrics.bytesWritten.toString.contains(searchValue) + || f.taskMetrics.get.outputMetrics.recordsWritten.toString.contains(searchValue) + || f.taskMetrics.get.shuffleWriteMetrics.bytesWritten.toString.contains(searchValue) + || f.taskMetrics.get.shuffleWriteMetrics.recordsWritten.toString.contains(searchValue) + || f.taskMetrics.get.shuffleWriteMetrics.writeTime.toString.contains(searchValue) + || f.schedulerDelay.toString.contains(searchValue) || f.gettingResultTime.toString.contains(searchValue))) + filteredTaskDataSequence } def executorSummary(stageId: Int, attemptId: Int): Map[String, v1.ExecutorStageSummary] = { diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala index 5362ad08bf349..0c17b6f1cdc43 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala @@ -19,7 +19,7 @@ package org.apache.spark.status.api.v1 import java.util import java.util.{Collections, Comparator, List => JList} import javax.ws.rs._ -import javax.ws.rs.core.{Context, MediaType, UriInfo} +import javax.ws.rs.core.{Context, MediaType, MultivaluedMap, UriInfo} import org.apache.spark.SparkException import org.apache.spark.scheduler.StageInfo @@ -125,29 +125,27 @@ private[v1] class StagesResource extends BaseAppResource { @Path("{stageId: \\d+}/taskTable") def taskTable( @PathParam("stageId") stageId: Int, - @QueryParam("details") @DefaultValue("true") details: Boolean, @Context uriInfo: UriInfo): util.HashMap[String, Seq[TaskData]] = { + @QueryParam("details") @DefaultValue("true") details: Boolean, @Context uriInfo: UriInfo): util.HashMap[String, Object] = { withUI { ui => val abc = uriInfo.getQueryParameters(true) + val totalRecords = abc.getFirst("numTasks") + var isSearch = false + var searchValue: String = null val iter = abc.keySet().iterator() while(iter.hasNext) { var ac = iter.next() System.err.println("hereeeeeeeeeeee 1 " + ac+" : "+abc.get(ac)) } - val queryParams = abc.keySet() - var columnToSort = 0 - if (queryParams.contains("order[0][column]")) { - columnToSort = abc.getFirst("order[0][column]").toInt - } - var columnNameToSort = abc.getFirst("columns["+columnToSort+"][name]") - if (columnNameToSort.equalsIgnoreCase("Logs")) { - columnNameToSort = "Index" - columnToSort = 0 + var _tasksToShow: Seq[TaskData] = null + if (abc.getFirst("search[value]") != null && abc.getFirst("search[value]").length > 0) { + _tasksToShow = ui.store.taskList(stageId, 0, 0, totalRecords.toInt, + indexName("Index"), true) + isSearch = true + searchValue = abc.getFirst("search[value]") + } else { + _tasksToShow = doPagination(abc, stageId) } - val isAscendingStr = abc.getFirst("order[0][dir]") - val _tasksToShow: Seq[TaskData] = ui.store.fullTaskList(stageId, 0, - indexName(columnNameToSort), isAscendingStr.equalsIgnoreCase("asc")) if (_tasksToShow.nonEmpty) { - val iterator = _tasksToShow.iterator while(iterator.hasNext) { val t1: TaskData = iterator.next() @@ -157,8 +155,29 @@ private[v1] class StagesResource extends BaseAppResource { t1.schedulerDelay = AppStatusUtils.schedulerDelay(t1) t1.gettingResultTime = AppStatusUtils.gettingResultTime(t1) } - val ret5 = new util.HashMap[String, Seq[TaskData]]() - ret5.put("aaData", _tasksToShow) + val ret5 = new util.HashMap[String, Object]() + if (isSearch) { + val filteredTaskList = ui.store.filterTaskList(_tasksToShow, searchValue) + if (filteredTaskList.length > 0) { + ret5.put("aaData", filteredTaskList) + } else { + _tasksToShow = doPagination(abc, stageId) + val iterator = _tasksToShow.iterator + while(iterator.hasNext) { + val t1: TaskData = iterator.next() + val execId = t1.executorId + val executorLogs = ui.store.executorSummary(execId).executorLogs + t1.executorLogs = executorLogs + t1.schedulerDelay = AppStatusUtils.schedulerDelay(t1) + t1.gettingResultTime = AppStatusUtils.gettingResultTime(t1) + } + ret5.put("aaData", _tasksToShow) + } + } else { + ret5.put("aaData", _tasksToShow) + } + ret5.put("recordsTotal", totalRecords) + ret5.put("recordsFiltered", totalRecords) ret5 } else { throw new NotFoundException(s"unknown stage: $stageId") @@ -166,4 +185,22 @@ private[v1] class StagesResource extends BaseAppResource { } } + def doPagination(queryParameters: MultivaluedMap[String, String], stageId: Int): Seq[TaskData] = { + val queryParams = queryParameters.keySet() + var columnToSort = 0 + if (queryParams.contains("order[0][column]")) { + columnToSort = queryParameters.getFirst("order[0][column]").toInt + } + var columnNameToSort = queryParameters.getFirst("columns[" + columnToSort + "][name]") + if (columnNameToSort.equalsIgnoreCase("Logs")) { + columnNameToSort = "Index" + columnToSort = 0 + } + val isAscendingStr = queryParameters.getFirst("order[0][dir]") + val pageStartIndex = queryParameters.getFirst("start").toInt + val pageLength = queryParameters.getFirst("length").toInt + return withUI(_.store.taskList(stageId, 0, pageStartIndex, pageLength, + indexName(columnNameToSort), isAscendingStr.equalsIgnoreCase("asc"))) + } + } diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/TaskDataComparator.scala b/core/src/main/scala/org/apache/spark/status/api/v1/TaskDataComparator.scala deleted file mode 100644 index d35e96d22daf3..0000000000000 --- a/core/src/main/scala/org/apache/spark/status/api/v1/TaskDataComparator.scala +++ /dev/null @@ -1,12 +0,0 @@ -package org.apache.spark.status.api.v1 - -import java.util.Comparator - -class TaskDataComparator(var index: Int) extends Comparator[TaskData] { - override def compare(o1: TaskData, o2: TaskData): Int = { - - if (o1.index == o2.index) return 0 - if (o1.index < o2.index) -1 - else 1 - } -} From 590e93f814c93f4ccc30af50d9f28ae92bb708e3 Mon Sep 17 00:00:00 2001 From: pgandhi Date: Sun, 1 Jul 2018 00:03:14 -0500 Subject: [PATCH 08/42] [SPARK-21809] : Fixing Tests and task metric table --- .../org/apache/spark/ui/static/stagepage.js | 428 +++++++++--------- .../org/apache/spark/ui/static/utils.js | 7 + .../spark/status/api/v1/StagesResource.scala | 18 +- .../blacklisting_for_stage_expectation.json | 48 +- ...acklisting_node_for_stage_expectation.json | 56 ++- .../one_stage_attempt_json_expectation.json | 32 +- .../one_stage_json_expectation.json | 44 +- .../stage_task_list_expectation.json | 80 +++- ...multi_attempt_app_json_1__expectation.json | 32 +- ...multi_attempt_app_json_2__expectation.json | 32 +- ...k_list_w__offset___length_expectation.json | 200 ++++++-- ...stage_task_list_w__sortBy_expectation.json | 80 +++- ...tBy_short_names___runtime_expectation.json | 80 +++- ...rtBy_short_names__runtime_expectation.json | 80 +++- ...age_with_accumulable_json_expectation.json | 32 +- .../org/apache/spark/ui/StagePageSuite.scala | 12 - 16 files changed, 824 insertions(+), 437 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js index 1d52520821562..69bccc0f0e28a 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js @@ -67,29 +67,44 @@ function stageEndPoint(appId) { return location.origin + "/api/v1/applications/" + appId + "/stages/" + stageId; } -function sortNumber(a,b) { - return a - b; -} - -function sortRecords(a,b) { - var aSplit = a.split("/"); - var bSplit = b.split("/"); - if (aSplit[0] == bSplit[0]) { - return aSplit[1] - bSplit[1]; - } - return aSplit[0] - bSplit[0]; -} - -function quantile(array, percentile) { - index = percentile/100. * (array.length-1); - if (Math.floor(index) == index) { - result = array[index]; - } else { - var i = Math.floor(index); - fraction = index - i; - result = array[i]; +function getColumnNameForTaskMetricSummary(columnKey) { + if(columnKey == "executorRunTime") { + return "Duration"; + } + else if(columnKey == "jvmGcTime") { + return "GC Time"; + } + else if(columnKey == "gettingResultTime") { + return "Getting Result Time"; + } + else if(columnKey == "inputMetrics") { + return "Input Size / Records"; + } + else if(columnKey == "outputMetrics") { + return "Output Size / Records"; + } + else if(columnKey == "peakExecutionMemory") { + return "Peak Execution Memory"; + } + else if(columnKey == "resultSerializationTime") { + return "Result Serialization Time"; } - return result; + else if(columnKey == "schedulerDelay") { + return "Scheduler Delay"; + } + else if(columnKey == "diskBytesSpilled") { + return "Shuffle spill (disk)"; + } + else if(columnKey == "memoryBytesSpilled") { + return "Shuffle spill (memory)"; + } + else if(columnKey == "shuffleWriteMetrics") { + return "Shuffle Write Size / Records"; + } + else if(columnKey == "executorDeserializeTime") { + return "Task Deserialization Time"; + } + return "NA"; } $(document).ready(function () { @@ -102,13 +117,13 @@ $(document).ready(function () { "" + ""); tasksSummary = $("#active-tasks"); @@ -117,13 +132,6 @@ $(document).ready(function () { var endPoint = stageEndPoint(appId); $.getJSON(endPoint, function(response, status, jqXHR) { - // prepare data for tasks table - var indices = Object.keys(response[0].tasks); - var task_table = []; - indices.forEach(function (ix) { - task_table.push(response[0].tasks[parseInt(ix)]); - }); - // prepare data for task aggregated metrics table indices = Object.keys(response[0].executorSummary); var task_summary_table = []; @@ -132,99 +140,6 @@ $(document).ready(function () { task_summary_table.push(response[0].executorSummary[ix]); }); - // prepare data for task summary table - var durationSummary = []; - var schedulerDelaySummary = []; - var taskDeserializationSummary = []; - var gcTimeSummary = []; - var resultSerializationTimeSummary = []; - var gettingResultTimeSummary = []; - var peakExecutionMemorySummary = []; - var inputSizeRecordsSummary = []; - var outputSizeRecordsSummary = []; - var shuffleWriteSizeRecordsSummary = []; - var shuffleSpillMemorySummary = []; - var shuffleSpillDiskSummary = []; - - console.log("hereeeeeeeeee 1 "+task_table.length); - task_table.forEach(function (x){ - if ("taskMetrics" in x) { - durationSummary.push(x.taskMetrics.executorRunTime); - taskDeserializationSummary.push(x.taskMetrics.executorDeserializeTime); - resultSerializationTimeSummary.push(x.taskMetrics.resultSerializationTime); - gcTimeSummary.push(x.taskMetrics.jvmGcTime); - peakExecutionMemorySummary.push(x.taskMetrics.peakExecutionMemory); - if (x.taskMetrics.inputMetrics.bytesRead > 0) { - inputSizeRecordsSummary.push(x.taskMetrics.inputMetrics.bytesRead + "/" + x.taskMetrics.inputMetrics.recordsRead); - } - if (x.taskMetrics.outputMetrics.bytesWritten > 0) { - outputSizeRecordsSummary.push(x.taskMetrics.outputMetrics.bytesWritten + "/" + x.taskMetrics.outputMetrics.recordsWritten); - } - if (x.taskMetrics.shuffleWriteMetrics.bytesWritten > 0) { - shuffleWriteSizeRecordsSummary.push(x.taskMetrics.shuffleWriteMetrics.bytesWritten + "/" + x.taskMetrics.shuffleWriteMetrics.recordsWritten); - } - if (x.taskMetrics.memoryBytesSpilled > 0) { - shuffleSpillMemorySummary.push(x.taskMetrics.memoryBytesSpilled); - } - if (x.taskMetrics.diskBytesSpilled > 0) { - shuffleSpillDiskSummary.push(x.taskMetrics.diskBytesSpilled); - } - } - schedulerDelaySummary.push(x.schedulerDelay); - gettingResultTimeSummary.push(x.gettingResultTime); - }); - - var task_metrics_table = []; - var task_metrics_table_all = []; - var task_metrics_table_col = ["Duration", "Scheduler Delay", "Task Deserialization Time", "GC Time", "Result Serialization Time", "Getting Result Time", "Peak Execution Memory"]; - - task_metrics_table_all.push(durationSummary); - task_metrics_table_all.push(schedulerDelaySummary); - task_metrics_table_all.push(taskDeserializationSummary); - task_metrics_table_all.push(gcTimeSummary); - task_metrics_table_all.push(resultSerializationTimeSummary); - task_metrics_table_all.push(gettingResultTimeSummary); - task_metrics_table_all.push(peakExecutionMemorySummary); - if (inputSizeRecordsSummary.length > 0) { - task_metrics_table_all.push(inputSizeRecordsSummary); - task_metrics_table_col.push("Input Size / Records"); - } - if (outputSizeRecordsSummary.length > 0) { - task_metrics_table_all.push(outputSizeRecordsSummary); - task_metrics_table_col.push("Output Size / Records"); - } - if (shuffleWriteSizeRecordsSummary.length > 0) { - task_metrics_table_all.push(shuffleWriteSizeRecordsSummary); - task_metrics_table_col.push("Shuffle Write Size / Records"); - } - if (shuffleSpillMemorySummary.length > 0) { - task_metrics_table_all.push(shuffleSpillMemorySummary); - task_metrics_table_col.push("Shuffle spill (memory)"); - } - if (shuffleSpillDiskSummary.length > 0) { - task_metrics_table_all.push(shuffleSpillDiskSummary); - task_metrics_table_col.push("Shuffle spill (disk)"); - } - - for(i = 0; i < task_metrics_table_col.length; i++){ - var task_sort_table; - if (task_metrics_table_col[i] == 'Input Size / Records' || task_metrics_table_col[i] == 'Output Size / Records' - || task_metrics_table_col[i] == 'Shuffle Write Size / Records') { - task_sort_table = (task_metrics_table_all[i]).sort(sortRecords); - } else { - task_sort_table = (task_metrics_table_all[i]).sort(sortNumber); - } - var row = { - "metric": task_metrics_table_col[i], - "p0": quantile(task_sort_table, 0), - "p25": quantile(task_sort_table, 25), - "p50": quantile(task_sort_table, 50), - "p75": quantile(task_sort_table, 75), - "p100": quantile(task_sort_table, 100) - }; - task_metrics_table.push(row); - } - // prepare data for accumulatorUpdates var indices = Object.keys(response[0].accumulatorUpdates); var accumulator_table_all = []; @@ -241,7 +156,7 @@ $(document).ready(function () { }); // rendering the UI page - var data = {executors: response, "taskstable": task_table, "task_metrics_table": task_metrics_table}; + var data = {"executors": response}; $.get(createTemplateURI(appId, "stagespage"), function(template) { tasksSummary.append(Mustache.render($(template).filter("#stages-summary-template").html(), data)); @@ -255,92 +170,157 @@ $(document).ready(function () { $("#toggle-aggregatedMetrics").toggle(); }); - // building task summary table - var taskMetricsTable = "#summary-metrics-table"; - var task_conf = { - "data": task_metrics_table, - "columns": [ - {data : 'metric'}, - { - data: function (row, type) { - if (row.metric == 'Input Size / Records' || row.metric == 'Output Size / Records' - || row.metric == 'Shuffle Write Size / Records') { - var strarray = row.p0.split("/"); - var str = formatBytes(strarray[0], type) + " / " + strarray[1]; - return str; - } else { - return (row.metric == 'Peak Execution Memory' || row.metric == 'Shuffle spill (memory)' - || row.metric == 'Shuffle spill (disk)') ? formatBytes(row.p0, type) : (formatDuration(row.p0)); + var task_metrics_table = []; + var stageAttemptId = getStageAttemptId(); + $.getJSON(stageEndPoint(appId) + "/"+stageAttemptId+"/taskSummary?quantiles=0,0.25,0.5,0.75,1.0", function(response1, status, jqXHR) { + var taskMetricIndices = Object.keys(response1); + taskMetricIndices.forEach(function (ix) { + var columnName = getColumnNameForTaskMetricSummary(ix); + if (columnName != "NA") { + var row = { + "metric": columnName, + "data": response1[ix] + }; + task_metrics_table.push(row); + } + }); + + var taskMetricsTable = "#summary-metrics-table"; + var task_conf = { + "data": task_metrics_table, + "columns": [ + {data : 'metric'}, + { + data: function (row, type) { + if (row.metric == 'Input Size / Records') { + var str1arr = JSON.stringify(row.data.bytesRead).split("[")[1].split("]")[0].split(","); + var str2arr = JSON.stringify(row.data.recordsRead).split("[")[1].split("]")[0].split(","); + var str = formatBytes(str1arr[0], type) + " / " + str2arr[0]; + return str; + } else if (row.metric == 'Output Size / Records') { + var str1arr = JSON.stringify(row.data.bytesWritten).split("[")[1].split("]")[0].split(","); + var str2arr = JSON.stringify(row.data.recordsWritten).split("[")[1].split("]")[0].split(","); + var str = formatBytes(str1arr[0], type) + " / " + str2arr[0]; + return str; + } else if (row.metric == 'Shuffle Write Size / Records') { + var str1arr = JSON.stringify(row.data.writeBytes).split("[")[1].split("]")[0].split(","); + var str2arr = JSON.stringify(row.data.writeRecords).split("[")[1].split("]")[0].split(","); + var str = formatBytes(str1arr[0], type) + " / " + str2arr[0]; + return str; + } else { + return (row.metric == 'Peak Execution Memory' || row.metric == 'Shuffle spill (memory)' + || row.metric == 'Shuffle spill (disk)') ? formatBytes(row.data[0], type) : (formatDuration(row.data[0])); + } } - } - }, - { - data: function (row, type) { - if (row.metric == 'Input Size / Records' || row.metric == 'Output Size / Records' - || row.metric == 'Shuffle Write Size / Records') { - var strarray = row.p25.split("/"); - var str = formatBytes(strarray[0], type) + " / " + strarray[1]; - return str; - } else { - return (row.metric == 'Peak Execution Memory' || row.metric == 'Shuffle spill (memory)' - || row.metric == 'Shuffle spill (disk)') ? formatBytes(row.p25, type) : (formatDuration(row.p25)); + }, + { + data: function (row, type) { + if (row.metric == 'Input Size / Records') { + var str1arr = JSON.stringify(row.data.bytesRead).split("[")[1].split("]")[0].split(","); + var str2arr = JSON.stringify(row.data.recordsRead).split("[")[1].split("]")[0].split(","); + var str = formatBytes(str1arr[1], type) + " / " + str2arr[1]; + return str; + } else if (row.metric == 'Output Size / Records') { + var str1arr = JSON.stringify(row.data.bytesWritten).split("[")[1].split("]")[0].split(","); + var str2arr = JSON.stringify(row.data.recordsWritten).split("[")[1].split("]")[0].split(","); + var str = formatBytes(str1arr[1], type) + " / " + str2arr[1]; + return str; + } else if (row.metric == 'Shuffle Write Size / Records') { + var str1arr = JSON.stringify(row.data.writeBytes).split("[")[1].split("]")[0].split(","); + var str2arr = JSON.stringify(row.data.writeRecords).split("[")[1].split("]")[0].split(","); + var str = formatBytes(str1arr[1], type) + " / " + str2arr[1]; + return str; + } else { + return (row.metric == 'Peak Execution Memory' || row.metric == 'Shuffle spill (memory)' + || row.metric == 'Shuffle spill (disk)') ? formatBytes(row.data[1], type) : (formatDuration(row.data[1])); + } } - } - }, - { - data: function (row, type) { - if (row.metric == 'Input Size / Records' || row.metric == 'Output Size / Records' - || row.metric == 'Shuffle Write Size / Records') { - var strarray = row.p50.split("/"); - var str = formatBytes(strarray[0], type) + " / " + strarray[1]; - return str; - } else { - return (row.metric == 'Peak Execution Memory' || row.metric == 'Shuffle spill (memory)' - || row.metric == 'Shuffle spill (disk)') ? formatBytes(row.p50, type) : (formatDuration(row.p50)); + }, + { + data: function (row, type) { + if (row.metric == 'Input Size / Records') { + var str1arr = JSON.stringify(row.data.bytesRead).split("[")[1].split("]")[0].split(","); + var str2arr = JSON.stringify(row.data.recordsRead).split("[")[1].split("]")[0].split(","); + var str = formatBytes(str1arr[2], type) + " / " + str2arr[2]; + return str; + } else if (row.metric == 'Output Size / Records') { + var str1arr = JSON.stringify(row.data.bytesWritten).split("[")[1].split("]")[0].split(","); + var str2arr = JSON.stringify(row.data.recordsWritten).split("[")[1].split("]")[0].split(","); + var str = formatBytes(str1arr[2], type) + " / " + str2arr[2]; + return str; + } else if (row.metric == 'Shuffle Write Size / Records') { + var str1arr = JSON.stringify(row.data.writeBytes).split("[")[1].split("]")[0].split(","); + var str2arr = JSON.stringify(row.data.writeRecords).split("[")[1].split("]")[0].split(","); + var str = formatBytes(str1arr[2], type) + " / " + str2arr[2]; + return str; + } else { + return (row.metric == 'Peak Execution Memory' || row.metric == 'Shuffle spill (memory)' + || row.metric == 'Shuffle spill (disk)') ? formatBytes(row.data[2], type) : (formatDuration(row.data[2])); + } } - } - }, - { - data: function (row, type) { - if (row.metric == 'Input Size / Records' || row.metric == 'Output Size / Records' - || row.metric == 'Shuffle Write Size / Records') { - var strarray = row.p75.split("/"); - var str = formatBytes(strarray[0], type) + " / " + strarray[1]; - return str; - } else { - return (row.metric == 'Peak Execution Memory' || row.metric == 'Shuffle spill (memory)' - || row.metric == 'Shuffle spill (disk)') ? formatBytes(row.p75, type) : (formatDuration(row.p75)); + }, + { + data: function (row, type) { + if (row.metric == 'Input Size / Records') { + var str1arr = JSON.stringify(row.data.bytesRead).split("[")[1].split("]")[0].split(","); + var str2arr = JSON.stringify(row.data.recordsRead).split("[")[1].split("]")[0].split(","); + var str = formatBytes(str1arr[3], type) + " / " + str2arr[3]; + return str; + } else if (row.metric == 'Output Size / Records') { + var str1arr = JSON.stringify(row.data.bytesWritten).split("[")[1].split("]")[0].split(","); + var str2arr = JSON.stringify(row.data.recordsWritten).split("[")[1].split("]")[0].split(","); + var str = formatBytes(str1arr[3], type) + " / " + str2arr[3]; + return str; + } else if (row.metric == 'Shuffle Write Size / Records') { + var str1arr = JSON.stringify(row.data.writeBytes).split("[")[1].split("]")[0].split(","); + var str2arr = JSON.stringify(row.data.writeRecords).split("[")[1].split("]")[0].split(","); + var str = formatBytes(str1arr[3], type) + " / " + str2arr[3]; + return str; + } else { + return (row.metric == 'Peak Execution Memory' || row.metric == 'Shuffle spill (memory)' + || row.metric == 'Shuffle spill (disk)') ? formatBytes(row.data[3], type) : (formatDuration(row.data[3])); + } } - } - }, - { - data: function (row, type) { - if (row.metric == 'Input Size / Records' || row.metric == 'Output Size / Records' - || row.metric == 'Shuffle Write Size / Records') { - var strarray = row.p100.split("/"); - var str = formatBytes(strarray[0], type) + " / " + strarray[1]; - return str; - } else { - return (row.metric == 'Peak Execution Memory' || row.metric == 'Shuffle spill (memory)' - || row.metric == 'Shuffle spill (disk)') ? formatBytes(row.p100, type) : (formatDuration(row.p100)); + }, + { + data: function (row, type) { + if (row.metric == 'Input Size / Records') { + var str1arr = JSON.stringify(row.data.bytesRead).split("[")[1].split("]")[0].split(","); + var str2arr = JSON.stringify(row.data.recordsRead).split("[")[1].split("]")[0].split(","); + var str = formatBytes(str1arr[4], type) + " / " + str2arr[4]; + return str; + } else if (row.metric == 'Output Size / Records') { + var str1arr = JSON.stringify(row.data.bytesWritten).split("[")[1].split("]")[0].split(","); + var str2arr = JSON.stringify(row.data.recordsWritten).split("[")[1].split("]")[0].split(","); + var str = formatBytes(str1arr[4], type) + " / " + str2arr[4]; + return str; + } else if (row.metric == 'Shuffle Write Size / Records') { + var str1arr = JSON.stringify(row.data.writeBytes).split("[")[1].split("]")[0].split(","); + var str2arr = JSON.stringify(row.data.writeRecords).split("[")[1].split("]")[0].split(","); + var str = formatBytes(str1arr[4], type) + " / " + str2arr[4]; + return str; + } else { + return (row.metric == 'Peak Execution Memory' || row.metric == 'Shuffle spill (memory)' + || row.metric == 'Shuffle spill (disk)') ? formatBytes(row.data[4], type) : (formatDuration(row.data[4])); + } } } - } - ], - "columnDefs": [ - { "type": "file-size", "targets": 1 }, - { "type": "file-size", "targets": 2 }, - { "type": "file-size", "targets": 3 }, - { "type": "file-size", "targets": 4 }, - { "type": "file-size", "targets": 5 } - ], - "paging": false, - "searching": false, - "order": [[0, "asc"]] - }; - $(taskMetricsTable).DataTable(task_conf); + ], + "columnDefs": [ + { "type": "file-size", "targets": 1 }, + { "type": "file-size", "targets": 2 }, + { "type": "file-size", "targets": 3 }, + { "type": "file-size", "targets": 4 }, + { "type": "file-size", "targets": 5 } + ], + "paging": false, + "searching": false, + "order": [[0, "asc"]] + }; + $(taskMetricsTable).DataTable(task_conf); + }); - // building task aggregated metric table + // building task aggregated metric table var tasksSummarytable = "#summary-stages-table"; var task_summary_conf = { "data": task_summary_table, @@ -414,6 +394,13 @@ $(document).ready(function () { } $(accumulatorTable).DataTable(accumulator_conf); + var showInputMetrics = false; + var showOutputMetrics = false; + var showShuffleWriteTimeMetrics = false; + var showShuffleWriteBytesMetrics = false; + var showDiskSpill = false; + var showMemSpill = false; + // building tasks table var taskTable = "#active-tasks-table"; var task_conf = { @@ -422,7 +409,7 @@ $(document).ready(function () { "info": true, "processing": true, "ajax": { - "url": stageEndPoint(appId) + "/taskTable", + "url": stageEndPoint(appId) + "/" + stageAttemptId + "/taskTable", "data": { "numTasks": response[0].numTasks }, @@ -535,7 +522,8 @@ $(document).ready(function () { { data : function (row, type) { if (accumulator_table.length > 0 && row.accumulatorUpdates.length > 0) { - return row.accumulatorUpdates[0].name + ' : ' + row.accumulatorUpdates[0].update; + var accIndex = row.accumulatorUpdates.length - 1; + return row.accumulatorUpdates[accIndex].name + ' : ' + row.accumulatorUpdates[accIndex].update; } else { return ""; } @@ -546,6 +534,7 @@ $(document).ready(function () { data : function (row, type) { if ("taskMetrics" in row) { if (row.taskMetrics.inputMetrics.bytesRead > 0) { + showInputMetrics = true; if (type === 'display') { return formatBytes(row.taskMetrics.inputMetrics.bytesRead, type) + " / " + row.taskMetrics.inputMetrics.recordsRead; } else { @@ -564,6 +553,7 @@ $(document).ready(function () { data : function (row, type) { if ("taskMetrics" in row) { if (row.taskMetrics.outputMetrics.bytesWritten > 0) { + showOutputMetrics = true; if (type === 'display') { return formatBytes(row.taskMetrics.outputMetrics.bytesWritten, type) + " / " + row.taskMetrics.outputMetrics.recordsWritten; } else { @@ -582,6 +572,7 @@ $(document).ready(function () { data : function (row, type) { if ("taskMetrics" in row) { if (row.taskMetrics.shuffleWriteMetrics.writeTime > 0) { + showShuffleWriteTimeMetrics = true; return type === 'display' ? formatDuration(parseInt(row.taskMetrics.shuffleWriteMetrics.writeTime) / 1000000) : row.taskMetrics.shuffleWriteMetrics.writeTime; } else { return ""; @@ -596,6 +587,7 @@ $(document).ready(function () { data : function (row, type) { if ("taskMetrics" in row) { if (row.taskMetrics.shuffleWriteMetrics.bytesWritten > 0) { + showShuffleWriteBytesMetrics = true; if (type === 'display') { return formatBytes(row.taskMetrics.shuffleWriteMetrics.bytesWritten, type) + " / " + row.taskMetrics.shuffleWriteMetrics.recordsWritten; } else { @@ -614,6 +606,7 @@ $(document).ready(function () { data : function (row, type) { if ("taskMetrics" in row) { if (row.taskMetrics.memoryBytesSpilled > 0) { + showMemSpill = true; return type === 'display' ? formatBytes(row.taskMetrics.memoryBytesSpilled, type) : row.taskMetrics.memoryBytesSpilled; } else { return ""; @@ -628,6 +621,7 @@ $(document).ready(function () { data : function (row, type) { if ("taskMetrics" in row) { if (row.taskMetrics.diskBytesSpilled > 0) { + showDiskSpill = true; return type === 'display' ? formatBytes(row.taskMetrics.diskBytesSpilled, type) : row.taskMetrics.diskBytesSpilled; } else { return ""; @@ -723,22 +717,22 @@ $(document).ready(function () { $("#accumulator-update-table").show(); } - if (inputSizeRecordsSummary.length > 0) { + if (showInputMetrics) { taskTableSelector.column(19).visible(true); } - if (outputSizeRecordsSummary.length > 0) { + if (showOutputMetrics) { taskTableSelector.column(20).visible(true); } - if (shuffleWriteSizeRecordsSummary.length > 0) { + if (showShuffleWriteTimeMetrics) { taskTableSelector.column(21).visible(true); } - if (shuffleWriteSizeRecordsSummary.length > 0) { + if (showShuffleWriteBytesMetrics) { taskTableSelector.column(22).visible(true); } - if (shuffleSpillMemorySummary.length > 0) { + if (showMemSpill) { taskTableSelector.column(23).visible(true); } - if (shuffleSpillDiskSummary.length > 0) { + if (showDiskSpill) { taskTableSelector.column(24).visible(true); } }); diff --git a/core/src/main/resources/org/apache/spark/ui/static/utils.js b/core/src/main/resources/org/apache/spark/ui/static/utils.js index f53d8ae6adef0..70f5d9e0e5fee 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/utils.js +++ b/core/src/main/resources/org/apache/spark/ui/static/utils.js @@ -110,6 +110,13 @@ function getStandAloneAppId(cb) { }); } +function getStageAttemptId() { + var words = document.baseURI.split('?'); + var attemptIdStr = words[1].split('&')[1]; + var stgAttemptId = attemptIdStr.substr(attemptIdStr.indexOf('=') + 1, attemptIdStr.length - 1); + return stgAttemptId; +} + // This function is a helper function for sorting in datatable. // When the data is in duration (e.g. 12ms 2s 2min 2h ) // It will convert the string into integer for correct ordering diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala index 0c17b6f1cdc43..8295c8b97aadf 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala @@ -122,28 +122,24 @@ private[v1] class StagesResource extends BaseAppResource { } @GET - @Path("{stageId: \\d+}/taskTable") + @Path("{stageId: \\d+}/{stageAttemptId: \\d+}/taskTable") def taskTable( @PathParam("stageId") stageId: Int, + @PathParam("stageAttemptId") stageAttemptId: Int, @QueryParam("details") @DefaultValue("true") details: Boolean, @Context uriInfo: UriInfo): util.HashMap[String, Object] = { withUI { ui => val abc = uriInfo.getQueryParameters(true) val totalRecords = abc.getFirst("numTasks") var isSearch = false var searchValue: String = null - val iter = abc.keySet().iterator() - while(iter.hasNext) { - var ac = iter.next() - System.err.println("hereeeeeeeeeeee 1 " + ac+" : "+abc.get(ac)) - } var _tasksToShow: Seq[TaskData] = null if (abc.getFirst("search[value]") != null && abc.getFirst("search[value]").length > 0) { - _tasksToShow = ui.store.taskList(stageId, 0, 0, totalRecords.toInt, + _tasksToShow = ui.store.taskList(stageId, stageAttemptId, 0, totalRecords.toInt, indexName("Index"), true) isSearch = true searchValue = abc.getFirst("search[value]") } else { - _tasksToShow = doPagination(abc, stageId) + _tasksToShow = doPagination(abc, stageId, stageAttemptId) } if (_tasksToShow.nonEmpty) { val iterator = _tasksToShow.iterator @@ -161,7 +157,7 @@ private[v1] class StagesResource extends BaseAppResource { if (filteredTaskList.length > 0) { ret5.put("aaData", filteredTaskList) } else { - _tasksToShow = doPagination(abc, stageId) + _tasksToShow = doPagination(abc, stageId, stageAttemptId) val iterator = _tasksToShow.iterator while(iterator.hasNext) { val t1: TaskData = iterator.next() @@ -185,7 +181,7 @@ private[v1] class StagesResource extends BaseAppResource { } } - def doPagination(queryParameters: MultivaluedMap[String, String], stageId: Int): Seq[TaskData] = { + def doPagination(queryParameters: MultivaluedMap[String, String], stageId: Int, stageAttemptId: Int): Seq[TaskData] = { val queryParams = queryParameters.keySet() var columnToSort = 0 if (queryParams.contains("order[0][column]")) { @@ -199,7 +195,7 @@ private[v1] class StagesResource extends BaseAppResource { val isAscendingStr = queryParameters.getFirst("order[0][dir]") val pageStartIndex = queryParameters.getFirst("start").toInt val pageLength = queryParameters.getFirst("length").toInt - return withUI(_.store.taskList(stageId, 0, pageStartIndex, pageLength, + return withUI(_.store.taskList(stageId, stageAttemptId, pageStartIndex, pageLength, indexName(columnNameToSort), isAscendingStr.equalsIgnoreCase("asc"))) } diff --git a/core/src/test/resources/HistoryServerExpectations/blacklisting_for_stage_expectation.json b/core/src/test/resources/HistoryServerExpectations/blacklisting_for_stage_expectation.json index 5e9e8230e2745..9d1e3e572914d 100644 --- a/core/src/test/resources/HistoryServerExpectations/blacklisting_for_stage_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/blacklisting_for_stage_expectation.json @@ -78,7 +78,9 @@ "writeTime": 3873006, "recordsWritten": 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "5": { "taskId": 5, @@ -125,7 +127,9 @@ "writeTime": 262919, "recordsWritten": 1 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "10": { "taskId": 10, @@ -172,7 +176,9 @@ "writeTime": 243647, "recordsWritten": 1 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "1": { "taskId": 1, @@ -219,7 +225,9 @@ "writeTime": 2409488, "recordsWritten": 1 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "6": { "taskId": 6, @@ -266,7 +274,9 @@ "writeTime": 385110, "recordsWritten": 1 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "9": { "taskId": 9, @@ -313,7 +323,9 @@ "writeTime": 259354, "recordsWritten": 1 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "2": { "taskId": 2, @@ -361,7 +373,9 @@ "writeTime": 126128, "recordsWritten": 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "7": { "taskId": 7, @@ -408,7 +422,9 @@ "writeTime": 205520, "recordsWritten": 1 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "3": { "taskId": 3, @@ -455,7 +471,9 @@ "writeTime": 207014, "recordsWritten": 1 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "11": { "taskId": 11, @@ -502,7 +520,9 @@ "writeTime": 233652, "recordsWritten": 1 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "8": { "taskId": 8, @@ -549,7 +569,9 @@ "writeTime": 213296, "recordsWritten": 1 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "4": { "taskId": 4, @@ -596,7 +618,9 @@ "writeTime": 292381, "recordsWritten": 1 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 } }, "executorSummary": { diff --git a/core/src/test/resources/HistoryServerExpectations/blacklisting_node_for_stage_expectation.json b/core/src/test/resources/HistoryServerExpectations/blacklisting_node_for_stage_expectation.json index acd4cc53de6cd..d6539f8305082 100644 --- a/core/src/test/resources/HistoryServerExpectations/blacklisting_node_for_stage_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/blacklisting_node_for_stage_expectation.json @@ -74,7 +74,9 @@ "writeTime" : 3662221, "recordsWritten" : 3 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "5" : { "taskId" : 5, @@ -122,7 +124,9 @@ "writeTime" : 191901, "recordsWritten" : 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "10" : { "taskId" : 10, @@ -169,7 +173,9 @@ "writeTime" : 301705, "recordsWritten" : 3 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "1" : { "taskId" : 1, @@ -217,7 +223,9 @@ "writeTime" : 3075188, "recordsWritten" : 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "6" : { "taskId" : 6, @@ -265,7 +273,9 @@ "writeTime" : 183718, "recordsWritten" : 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "9" : { "taskId" : 9, @@ -312,7 +322,9 @@ "writeTime" : 366050, "recordsWritten" : 3 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "13" : { "taskId" : 13, @@ -359,7 +371,9 @@ "writeTime" : 369513, "recordsWritten" : 3 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "2" : { "taskId" : 2, @@ -406,7 +420,9 @@ "writeTime" : 3322956, "recordsWritten" : 3 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "12" : { "taskId" : 12, @@ -453,7 +469,9 @@ "writeTime" : 319101, "recordsWritten" : 3 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "7" : { "taskId" : 7, @@ -500,7 +518,9 @@ "writeTime" : 377601, "recordsWritten" : 3 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "3" : { "taskId" : 3, @@ -547,7 +567,9 @@ "writeTime" : 3587839, "recordsWritten" : 3 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "11" : { "taskId" : 11, @@ -594,7 +616,9 @@ "writeTime" : 323898, "recordsWritten" : 3 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "8" : { "taskId" : 8, @@ -641,7 +665,9 @@ "writeTime" : 311940, "recordsWritten" : 3 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "4" : { "taskId" : 4, @@ -689,7 +715,9 @@ "writeTime" : 16858066, "recordsWritten" : 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 } }, "executorSummary" : { diff --git a/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json index 03f886afa5413..10bf89c8ffd55 100644 --- a/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json @@ -74,7 +74,9 @@ "writeTime" : 76000, "recordsWritten" : 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "14" : { "taskId" : 14, @@ -121,7 +123,9 @@ "writeTime" : 88000, "recordsWritten" : 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "9" : { "taskId" : 9, @@ -168,7 +172,9 @@ "writeTime" : 98000, "recordsWritten" : 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "13" : { "taskId" : 13, @@ -215,7 +221,9 @@ "writeTime" : 73000, "recordsWritten" : 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "12" : { "taskId" : 12, @@ -262,7 +270,9 @@ "writeTime" : 101000, "recordsWritten" : 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "11" : { "taskId" : 11, @@ -309,7 +319,9 @@ "writeTime" : 83000, "recordsWritten" : 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "8" : { "taskId" : 8, @@ -356,7 +368,9 @@ "writeTime" : 94000, "recordsWritten" : 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "15" : { "taskId" : 15, @@ -403,7 +417,9 @@ "writeTime" : 79000, "recordsWritten" : 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 } }, "executorSummary" : { diff --git a/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json index 947c89906955d..3f44d2cd6b43b 100644 --- a/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json @@ -74,7 +74,10 @@ "writeTime" : 76000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 19, + "gettingResultTime" : 0 }, "14" : { "taskId" : 14, @@ -121,7 +124,10 @@ "writeTime" : 88000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 13, + "gettingResultTime" : 0 }, "9" : { "taskId" : 9, @@ -168,7 +174,10 @@ "writeTime" : 98000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 17, + "gettingResultTime" : 0 }, "13" : { "taskId" : 13, @@ -215,7 +224,10 @@ "writeTime" : 73000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 14, + "gettingResultTime" : 0 }, "12" : { "taskId" : 12, @@ -262,7 +274,10 @@ "writeTime" : 101000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 17, + "gettingResultTime" : 0 }, "11" : { "taskId" : 11, @@ -309,7 +324,10 @@ "writeTime" : 83000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 17, + "gettingResultTime" : 0 }, "8" : { "taskId" : 8, @@ -356,7 +374,10 @@ "writeTime" : 94000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 16, + "gettingResultTime" : 0 }, "15" : { "taskId" : 15, @@ -403,7 +424,10 @@ "writeTime" : 79000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 13, + "gettingResultTime" : 0 } }, "executorSummary" : { @@ -422,7 +446,9 @@ "shuffleWriteRecords" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, - "isBlacklistedForStage" : false + "isBlacklistedForStage" : false, + "executorLogs" : { }, + "hostPort" : "localhost:57971" } }, "killedTasksSummary" : { } diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_expectation.json index a15ee23523365..443bfb7ccf282 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_expectation.json @@ -43,7 +43,9 @@ "writeTime" : 3842811, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 1, "index" : 1, @@ -89,7 +91,9 @@ "writeTime" : 3934399, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 2, "index" : 2, @@ -135,7 +139,9 @@ "writeTime" : 89885, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 3, "index" : 3, @@ -181,7 +187,9 @@ "writeTime" : 1311694, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 4, "index" : 4, @@ -227,7 +235,9 @@ "writeTime" : 83022, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 5, "index" : 5, @@ -273,7 +283,9 @@ "writeTime" : 3675510, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 6, "index" : 6, @@ -319,7 +331,9 @@ "writeTime" : 4016617, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 7, "index" : 7, @@ -365,7 +379,9 @@ "writeTime" : 2579051, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 8, "index" : 8, @@ -411,7 +427,9 @@ "writeTime" : 121551, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 9, "index" : 9, @@ -457,7 +475,9 @@ "writeTime" : 101664, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 10, "index" : 10, @@ -503,7 +523,9 @@ "writeTime" : 94709, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 11, "index" : 11, @@ -549,7 +571,9 @@ "writeTime" : 94507, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 12, "index" : 12, @@ -595,7 +619,9 @@ "writeTime" : 102476, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 13, "index" : 13, @@ -641,7 +667,9 @@ "writeTime" : 95004, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 14, "index" : 14, @@ -687,7 +715,9 @@ "writeTime" : 95646, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 15, "index" : 15, @@ -733,7 +763,9 @@ "writeTime" : 602780, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 16, "index" : 16, @@ -779,7 +811,9 @@ "writeTime" : 108320, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 17, "index" : 17, @@ -825,7 +859,9 @@ "writeTime" : 99944, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 18, "index" : 18, @@ -871,7 +907,9 @@ "writeTime" : 100836, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 19, "index" : 19, @@ -917,5 +955,7 @@ "writeTime" : 95788, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 } ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_1__expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_1__expectation.json index f9182b1658334..c11103c3ed796 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_1__expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_1__expectation.json @@ -48,7 +48,9 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 1, "index" : 1, @@ -99,7 +101,9 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 2, "index" : 2, @@ -150,7 +154,9 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 3, "index" : 3, @@ -201,7 +207,9 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 4, "index" : 4, @@ -252,7 +260,9 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 5, "index" : 5, @@ -303,7 +313,9 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 6, "index" : 6, @@ -354,7 +366,9 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 7, "index" : 7, @@ -405,5 +419,7 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 } ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_2__expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_2__expectation.json index 76dd2f710b90f..5c055f8cdaab3 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_2__expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_2__expectation.json @@ -48,7 +48,9 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 1, "index" : 1, @@ -99,7 +101,9 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 2, "index" : 2, @@ -150,7 +154,9 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 3, "index" : 3, @@ -201,7 +207,9 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 4, "index" : 4, @@ -252,7 +260,9 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 5, "index" : 5, @@ -303,7 +313,9 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 6, "index" : 6, @@ -354,7 +366,9 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 7, "index" : 7, @@ -405,5 +419,7 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 } ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__offset___length_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__offset___length_expectation.json index 6bdc10465d89e..17ee461f89eae 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__offset___length_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__offset___length_expectation.json @@ -43,7 +43,9 @@ "writeTime" : 94709, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 11, "index" : 11, @@ -89,7 +91,9 @@ "writeTime" : 94507, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 12, "index" : 12, @@ -135,7 +139,9 @@ "writeTime" : 102476, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 13, "index" : 13, @@ -181,7 +187,9 @@ "writeTime" : 95004, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 14, "index" : 14, @@ -227,7 +235,9 @@ "writeTime" : 95646, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 15, "index" : 15, @@ -273,7 +283,9 @@ "writeTime" : 602780, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 16, "index" : 16, @@ -319,7 +331,9 @@ "writeTime" : 108320, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 17, "index" : 17, @@ -365,7 +379,9 @@ "writeTime" : 99944, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 18, "index" : 18, @@ -411,7 +427,9 @@ "writeTime" : 100836, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 19, "index" : 19, @@ -457,7 +475,9 @@ "writeTime" : 95788, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 20, "index" : 20, @@ -503,7 +523,9 @@ "writeTime" : 97716, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 21, "index" : 21, @@ -549,7 +571,9 @@ "writeTime" : 100270, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 22, "index" : 22, @@ -595,7 +619,9 @@ "writeTime" : 143427, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 23, "index" : 23, @@ -641,7 +667,9 @@ "writeTime" : 91844, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 24, "index" : 24, @@ -687,7 +715,9 @@ "writeTime" : 157194, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 25, "index" : 25, @@ -733,7 +763,9 @@ "writeTime" : 94134, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 26, "index" : 26, @@ -779,7 +811,9 @@ "writeTime" : 108213, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 27, "index" : 27, @@ -825,7 +859,9 @@ "writeTime" : 102019, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 28, "index" : 28, @@ -871,7 +907,9 @@ "writeTime" : 104299, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 29, "index" : 29, @@ -917,7 +955,9 @@ "writeTime" : 114938, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 30, "index" : 30, @@ -963,7 +1003,9 @@ "writeTime" : 119770, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 31, "index" : 31, @@ -1009,7 +1051,9 @@ "writeTime" : 92619, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 32, "index" : 32, @@ -1055,7 +1099,9 @@ "writeTime" : 89603, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 33, "index" : 33, @@ -1101,7 +1147,9 @@ "writeTime" : 118329, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 34, "index" : 34, @@ -1147,7 +1195,9 @@ "writeTime" : 127746, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 35, "index" : 35, @@ -1193,7 +1243,9 @@ "writeTime" : 160963, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 36, "index" : 36, @@ -1239,7 +1291,9 @@ "writeTime" : 123855, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 37, "index" : 37, @@ -1285,7 +1339,9 @@ "writeTime" : 111869, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 38, "index" : 38, @@ -1331,7 +1387,9 @@ "writeTime" : 131158, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 39, "index" : 39, @@ -1377,7 +1435,9 @@ "writeTime" : 98748, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 40, "index" : 40, @@ -1423,7 +1483,9 @@ "writeTime" : 94792, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 41, "index" : 41, @@ -1469,7 +1531,9 @@ "writeTime" : 90765, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 42, "index" : 42, @@ -1515,7 +1579,9 @@ "writeTime" : 103713, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 43, "index" : 43, @@ -1561,7 +1627,9 @@ "writeTime" : 171516, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 44, "index" : 44, @@ -1607,7 +1675,9 @@ "writeTime" : 98293, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 45, "index" : 45, @@ -1653,7 +1723,9 @@ "writeTime" : 92985, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 46, "index" : 46, @@ -1699,7 +1771,9 @@ "writeTime" : 113322, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 47, "index" : 47, @@ -1745,7 +1819,9 @@ "writeTime" : 103015, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 48, "index" : 48, @@ -1791,7 +1867,9 @@ "writeTime" : 139844, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 49, "index" : 49, @@ -1837,7 +1915,9 @@ "writeTime" : 94984, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 50, "index" : 50, @@ -1883,7 +1963,9 @@ "writeTime" : 90836, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 51, "index" : 51, @@ -1929,7 +2011,9 @@ "writeTime" : 96013, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 52, "index" : 52, @@ -1975,7 +2059,9 @@ "writeTime" : 89664, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 53, "index" : 53, @@ -2021,7 +2107,9 @@ "writeTime" : 92835, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 54, "index" : 54, @@ -2067,7 +2155,9 @@ "writeTime" : 90506, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 55, "index" : 55, @@ -2113,7 +2203,9 @@ "writeTime" : 108309, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 56, "index" : 56, @@ -2159,7 +2251,9 @@ "writeTime" : 90329, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 57, "index" : 57, @@ -2205,7 +2299,9 @@ "writeTime" : 96849, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 58, "index" : 58, @@ -2251,7 +2347,9 @@ "writeTime" : 97521, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 59, "index" : 59, @@ -2297,5 +2395,7 @@ "writeTime" : 100753, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 } ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_expectation.json index bc1cd49909d31..3c50553c12398 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_expectation.json @@ -43,7 +43,9 @@ "writeTime" : 4016617, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 5, "index" : 5, @@ -89,7 +91,9 @@ "writeTime" : 3675510, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 1, "index" : 1, @@ -135,7 +139,9 @@ "writeTime" : 3934399, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 7, "index" : 7, @@ -181,7 +187,9 @@ "writeTime" : 2579051, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 4, "index" : 4, @@ -227,7 +235,9 @@ "writeTime" : 83022, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 3, "index" : 3, @@ -273,7 +283,9 @@ "writeTime" : 1311694, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 0, "index" : 0, @@ -319,7 +331,9 @@ "writeTime" : 3842811, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 2, "index" : 2, @@ -365,7 +379,9 @@ "writeTime" : 89885, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 22, "index" : 22, @@ -411,7 +427,9 @@ "writeTime" : 143427, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 18, "index" : 18, @@ -457,7 +475,9 @@ "writeTime" : 100836, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 17, "index" : 17, @@ -503,7 +523,9 @@ "writeTime" : 99944, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 21, "index" : 21, @@ -549,7 +571,9 @@ "writeTime" : 100270, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 19, "index" : 19, @@ -595,7 +619,9 @@ "writeTime" : 95788, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 16, "index" : 16, @@ -641,7 +667,9 @@ "writeTime" : 108320, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 9, "index" : 9, @@ -687,7 +715,9 @@ "writeTime" : 101664, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 20, "index" : 20, @@ -733,7 +763,9 @@ "writeTime" : 97716, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 14, "index" : 14, @@ -779,7 +811,9 @@ "writeTime" : 95646, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 8, "index" : 8, @@ -825,7 +859,9 @@ "writeTime" : 121551, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 12, "index" : 12, @@ -871,7 +907,9 @@ "writeTime" : 102476, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 15, "index" : 15, @@ -917,5 +955,7 @@ "writeTime" : 602780, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 } ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names___runtime_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names___runtime_expectation.json index bc1cd49909d31..3c50553c12398 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names___runtime_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names___runtime_expectation.json @@ -43,7 +43,9 @@ "writeTime" : 4016617, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 5, "index" : 5, @@ -89,7 +91,9 @@ "writeTime" : 3675510, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 1, "index" : 1, @@ -135,7 +139,9 @@ "writeTime" : 3934399, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 7, "index" : 7, @@ -181,7 +187,9 @@ "writeTime" : 2579051, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 4, "index" : 4, @@ -227,7 +235,9 @@ "writeTime" : 83022, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 3, "index" : 3, @@ -273,7 +283,9 @@ "writeTime" : 1311694, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 0, "index" : 0, @@ -319,7 +331,9 @@ "writeTime" : 3842811, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 2, "index" : 2, @@ -365,7 +379,9 @@ "writeTime" : 89885, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 22, "index" : 22, @@ -411,7 +427,9 @@ "writeTime" : 143427, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 18, "index" : 18, @@ -457,7 +475,9 @@ "writeTime" : 100836, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 17, "index" : 17, @@ -503,7 +523,9 @@ "writeTime" : 99944, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 21, "index" : 21, @@ -549,7 +571,9 @@ "writeTime" : 100270, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 19, "index" : 19, @@ -595,7 +619,9 @@ "writeTime" : 95788, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 16, "index" : 16, @@ -641,7 +667,9 @@ "writeTime" : 108320, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 9, "index" : 9, @@ -687,7 +715,9 @@ "writeTime" : 101664, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 20, "index" : 20, @@ -733,7 +763,9 @@ "writeTime" : 97716, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 14, "index" : 14, @@ -779,7 +811,9 @@ "writeTime" : 95646, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 8, "index" : 8, @@ -825,7 +859,9 @@ "writeTime" : 121551, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 12, "index" : 12, @@ -871,7 +907,9 @@ "writeTime" : 102476, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 15, "index" : 15, @@ -917,5 +955,7 @@ "writeTime" : 602780, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 } ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names__runtime_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names__runtime_expectation.json index 09857cb401acd..f26232157427b 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names__runtime_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names__runtime_expectation.json @@ -43,7 +43,9 @@ "writeTime" : 94792, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 41, "index" : 41, @@ -89,7 +91,9 @@ "writeTime" : 90765, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 43, "index" : 43, @@ -135,7 +139,9 @@ "writeTime" : 171516, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 57, "index" : 57, @@ -181,7 +187,9 @@ "writeTime" : 96849, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 58, "index" : 58, @@ -227,7 +235,9 @@ "writeTime" : 97521, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 68, "index" : 68, @@ -273,7 +283,9 @@ "writeTime" : 101750, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 86, "index" : 86, @@ -319,7 +331,9 @@ "writeTime" : 95848, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 32, "index" : 32, @@ -365,7 +379,9 @@ "writeTime" : 89603, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 39, "index" : 39, @@ -411,7 +427,9 @@ "writeTime" : 98748, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 42, "index" : 42, @@ -457,7 +475,9 @@ "writeTime" : 103713, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 51, "index" : 51, @@ -503,7 +523,9 @@ "writeTime" : 96013, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 59, "index" : 59, @@ -549,7 +571,9 @@ "writeTime" : 100753, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 63, "index" : 63, @@ -595,7 +619,9 @@ "writeTime" : 102779, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 87, "index" : 87, @@ -641,7 +667,9 @@ "writeTime" : 102159, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 90, "index" : 90, @@ -687,7 +715,9 @@ "writeTime" : 98472, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 99, "index" : 99, @@ -733,7 +763,9 @@ "writeTime" : 133964, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 44, "index" : 44, @@ -779,7 +811,9 @@ "writeTime" : 98293, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 47, "index" : 47, @@ -825,7 +859,9 @@ "writeTime" : 103015, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 50, "index" : 50, @@ -871,7 +907,9 @@ "writeTime" : 90836, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, { "taskId" : 52, "index" : 52, @@ -917,5 +955,7 @@ "writeTime" : 89664, "recordsWritten" : 10 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 } ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json index 963f010968b62..082d6b3738f6c 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json @@ -83,7 +83,9 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "1" : { "taskId" : 1, @@ -135,7 +137,9 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "2" : { "taskId" : 2, @@ -187,7 +191,9 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "3" : { "taskId" : 3, @@ -239,7 +245,9 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "4" : { "taskId" : 4, @@ -291,7 +299,9 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "5" : { "taskId" : 5, @@ -343,7 +353,9 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "6" : { "taskId" : 6, @@ -395,7 +407,9 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 }, "7" : { "taskId" : 7, @@ -447,7 +461,9 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "schedulerDelay" : 0, + "gettingResultTime" : 0 } }, "executorSummary" : { 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 6044563f7dde7..40dfe646310f7 100644 --- a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala @@ -96,18 +96,6 @@ class StagePageSuite extends SparkFunSuite with LocalSparkContext { } } - test("peak execution memory should displayed") { - val html = renderStagePage().toString().toLowerCase(Locale.ROOT) - val targetString = "peak execution memory" - assert(html.contains(targetString)) - } - - test("SPARK-10543: peak execution memory should be per-task rather than cumulative") { - val html = renderStagePage().toString().toLowerCase(Locale.ROOT) - // verify min/25/50/75/max show task value not cumulative values - assert(html.contains(s"$peakExecutionMemory.0 b" * 5)) - } - /** * Render a stage page started with the given conf and return the HTML. * This also runs a dummy stage to populate the page with useful content. From d79eb800bd8169cd063e1083d0444f755ffdb036 Mon Sep 17 00:00:00 2001 From: pgandhi Date: Sun, 1 Jul 2018 14:18:59 -0500 Subject: [PATCH 09/42] [SPARK-21809] : Refactoring Code and adding shufflereadmetrics data --- .../org/apache/spark/ui/static/stagepage.js | 150 +++++++++++++----- .../spark/ui/static/stagespage-template.html | 1 + .../apache/spark/status/AppStatusStore.scala | 3 + .../spark/status/api/v1/StagesResource.scala | 29 ++-- 4 files changed, 127 insertions(+), 56 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js index 69bccc0f0e28a..f515dbc773718 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js @@ -98,6 +98,9 @@ function getColumnNameForTaskMetricSummary(columnKey) { else if(columnKey == "memoryBytesSpilled") { return "Shuffle spill (memory)"; } + else if(columnKey == "shuffleReadMetrics") { + return "Shuffle Read Size / Records"; + } else if(columnKey == "shuffleWriteMetrics") { return "Shuffle Write Size / Records"; } @@ -176,7 +179,24 @@ $(document).ready(function () { var taskMetricIndices = Object.keys(response1); taskMetricIndices.forEach(function (ix) { var columnName = getColumnNameForTaskMetricSummary(ix); - if (columnName != "NA") { + if (columnName == "Shuffle Read Size / Records") { + var row1 = { + "metric": columnName, + "data": response1[ix] + }; + var row2 = { + "metric": "Shuffle Read Blocked Time", + "data": response1[ix] + }; + var row3 = { + "metric": "Shuffle Remote Reads", + "data": response1[ix] + }; + task_metrics_table.push(row1); + task_metrics_table.push(row2); + task_metrics_table.push(row3); + } + else if (columnName != "NA") { var row = { "metric": columnName, "data": response1[ix] @@ -202,6 +222,19 @@ $(document).ready(function () { var str2arr = JSON.stringify(row.data.recordsWritten).split("[")[1].split("]")[0].split(","); var str = formatBytes(str1arr[0], type) + " / " + str2arr[0]; return str; + } else if (row.metric == 'Shuffle Read Size / Records') { + var str1arr = JSON.stringify(row.data.readBytes).split("[")[1].split("]")[0].split(","); + var str2arr = JSON.stringify(row.data.readRecords).split("[")[1].split("]")[0].split(","); + var str = formatBytes(str1arr[0], type) + " / " + str2arr[0]; + return str; + } else if (row.metric == 'Shuffle Read Blocked Time') { + var str1arr = JSON.stringify(row.data.fetchWaitTime).split("[")[1].split("]")[0].split(","); + var str = formatDuration(str1arr[0]); + return str; + } else if (row.metric == 'Shuffle Remote Reads') { + var str1arr = JSON.stringify(row.data.remoteBytesRead).split("[")[1].split("]")[0].split(","); + var str = formatBytes(str1arr[0], type); + return str; } else if (row.metric == 'Shuffle Write Size / Records') { var str1arr = JSON.stringify(row.data.writeBytes).split("[")[1].split("]")[0].split(","); var str2arr = JSON.stringify(row.data.writeRecords).split("[")[1].split("]")[0].split(","); @@ -225,6 +258,19 @@ $(document).ready(function () { var str2arr = JSON.stringify(row.data.recordsWritten).split("[")[1].split("]")[0].split(","); var str = formatBytes(str1arr[1], type) + " / " + str2arr[1]; return str; + } else if (row.metric == 'Shuffle Read Size / Records') { + var str1arr = JSON.stringify(row.data.readBytes).split("[")[1].split("]")[0].split(","); + var str2arr = JSON.stringify(row.data.readRecords).split("[")[1].split("]")[0].split(","); + var str = formatBytes(str1arr[1], type) + " / " + str2arr[1]; + return str; + } else if (row.metric == 'Shuffle Read Blocked Time') { + var str1arr = JSON.stringify(row.data.fetchWaitTime).split("[")[1].split("]")[0].split(","); + var str = formatDuration(str1arr[1]); + return str; + } else if (row.metric == 'Shuffle Remote Reads') { + var str1arr = JSON.stringify(row.data.remoteBytesRead).split("[")[1].split("]")[0].split(","); + var str = formatBytes(str1arr[1], type); + return str; } else if (row.metric == 'Shuffle Write Size / Records') { var str1arr = JSON.stringify(row.data.writeBytes).split("[")[1].split("]")[0].split(","); var str2arr = JSON.stringify(row.data.writeRecords).split("[")[1].split("]")[0].split(","); @@ -248,6 +294,19 @@ $(document).ready(function () { var str2arr = JSON.stringify(row.data.recordsWritten).split("[")[1].split("]")[0].split(","); var str = formatBytes(str1arr[2], type) + " / " + str2arr[2]; return str; + } else if (row.metric == 'Shuffle Read Size / Records') { + var str1arr = JSON.stringify(row.data.readBytes).split("[")[1].split("]")[0].split(","); + var str2arr = JSON.stringify(row.data.readRecords).split("[")[1].split("]")[0].split(","); + var str = formatBytes(str1arr[2], type) + " / " + str2arr[2]; + return str; + } else if (row.metric == 'Shuffle Read Blocked Time') { + var str1arr = JSON.stringify(row.data.fetchWaitTime).split("[")[1].split("]")[0].split(","); + var str = formatDuration(str1arr[2]); + return str; + } else if (row.metric == 'Shuffle Remote Reads') { + var str1arr = JSON.stringify(row.data.remoteBytesRead).split("[")[1].split("]")[0].split(","); + var str = formatBytes(str1arr[2], type); + return str; } else if (row.metric == 'Shuffle Write Size / Records') { var str1arr = JSON.stringify(row.data.writeBytes).split("[")[1].split("]")[0].split(","); var str2arr = JSON.stringify(row.data.writeRecords).split("[")[1].split("]")[0].split(","); @@ -271,6 +330,19 @@ $(document).ready(function () { var str2arr = JSON.stringify(row.data.recordsWritten).split("[")[1].split("]")[0].split(","); var str = formatBytes(str1arr[3], type) + " / " + str2arr[3]; return str; + } else if (row.metric == 'Shuffle Read Size / Records') { + var str1arr = JSON.stringify(row.data.readBytes).split("[")[1].split("]")[0].split(","); + var str2arr = JSON.stringify(row.data.readRecords).split("[")[1].split("]")[0].split(","); + var str = formatBytes(str1arr[3], type) + " / " + str2arr[3]; + return str; + } else if (row.metric == 'Shuffle Read Blocked Time') { + var str1arr = JSON.stringify(row.data.fetchWaitTime).split("[")[1].split("]")[0].split(","); + var str = formatDuration(str1arr[3]); + return str; + } else if (row.metric == 'Shuffle Remote Reads') { + var str1arr = JSON.stringify(row.data.remoteBytesRead).split("[")[1].split("]")[0].split(","); + var str = formatBytes(str1arr[3], type); + return str; } else if (row.metric == 'Shuffle Write Size / Records') { var str1arr = JSON.stringify(row.data.writeBytes).split("[")[1].split("]")[0].split(","); var str2arr = JSON.stringify(row.data.writeRecords).split("[")[1].split("]")[0].split(","); @@ -294,7 +366,20 @@ $(document).ready(function () { var str2arr = JSON.stringify(row.data.recordsWritten).split("[")[1].split("]")[0].split(","); var str = formatBytes(str1arr[4], type) + " / " + str2arr[4]; return str; - } else if (row.metric == 'Shuffle Write Size / Records') { + } else if (row.metric == 'Shuffle Read Size / Records') { + var str1arr = JSON.stringify(row.data.readBytes).split("[")[1].split("]")[0].split(","); + var str2arr = JSON.stringify(row.data.readRecords).split("[")[1].split("]")[0].split(","); + var str = formatBytes(str1arr[4], type) + " / " + str2arr[4]; + return str; + } else if (row.metric == 'Shuffle Read Blocked Time') { + var str1arr = JSON.stringify(row.data.fetchWaitTime).split("[")[1].split("]")[0].split(","); + var str = formatDuration(str1arr[4]); + return str; + } else if (row.metric == 'Shuffle Remote Reads') { + var str1arr = JSON.stringify(row.data.remoteBytesRead).split("[")[1].split("]")[0].split(","); + var str = formatBytes(str1arr[4], type); + return str; + } else if (row.metric == 'Shuffle Write Size / Records') { var str1arr = JSON.stringify(row.data.writeBytes).split("[")[1].split("]")[0].split(","); var str2arr = JSON.stringify(row.data.writeRecords).split("[")[1].split("]")[0].split(","); var str = formatBytes(str1arr[4], type) + " / " + str2arr[4]; @@ -394,14 +479,7 @@ $(document).ready(function () { } $(accumulatorTable).DataTable(accumulator_conf); - var showInputMetrics = false; - var showOutputMetrics = false; - var showShuffleWriteTimeMetrics = false; - var showShuffleWriteBytesMetrics = false; - var showDiskSpill = false; - var showMemSpill = false; - - // building tasks table + // building tasks table that uses server side functionality var taskTable = "#active-tasks-table"; var task_conf = { "serverSide": true, @@ -534,7 +612,6 @@ $(document).ready(function () { data : function (row, type) { if ("taskMetrics" in row) { if (row.taskMetrics.inputMetrics.bytesRead > 0) { - showInputMetrics = true; if (type === 'display') { return formatBytes(row.taskMetrics.inputMetrics.bytesRead, type) + " / " + row.taskMetrics.inputMetrics.recordsRead; } else { @@ -553,7 +630,6 @@ $(document).ready(function () { data : function (row, type) { if ("taskMetrics" in row) { if (row.taskMetrics.outputMetrics.bytesWritten > 0) { - showOutputMetrics = true; if (type === 'display') { return formatBytes(row.taskMetrics.outputMetrics.bytesWritten, type) + " / " + row.taskMetrics.outputMetrics.recordsWritten; } else { @@ -572,7 +648,6 @@ $(document).ready(function () { data : function (row, type) { if ("taskMetrics" in row) { if (row.taskMetrics.shuffleWriteMetrics.writeTime > 0) { - showShuffleWriteTimeMetrics = true; return type === 'display' ? formatDuration(parseInt(row.taskMetrics.shuffleWriteMetrics.writeTime) / 1000000) : row.taskMetrics.shuffleWriteMetrics.writeTime; } else { return ""; @@ -587,7 +662,6 @@ $(document).ready(function () { data : function (row, type) { if ("taskMetrics" in row) { if (row.taskMetrics.shuffleWriteMetrics.bytesWritten > 0) { - showShuffleWriteBytesMetrics = true; if (type === 'display') { return formatBytes(row.taskMetrics.shuffleWriteMetrics.bytesWritten, type) + " / " + row.taskMetrics.shuffleWriteMetrics.recordsWritten; } else { @@ -602,11 +676,29 @@ $(document).ready(function () { }, name: "Shuffle Write Size / Records" }, + { + data : function (row, type) { + if ("taskMetrics" in row) { + if (row.taskMetrics.shuffleReadMetrics.localBytesRead > 0) { + var totalBytesRead = parseInt(row.taskMetrics.shuffleReadMetrics.localBytesRead) + parseInt(row.taskMetrics.shuffleReadMetrics.remoteBytesRead); + if (type === 'display') { + return formatBytes(totalBytesRead, type) + " / " + row.taskMetrics.shuffleReadMetrics.recordsRead; + } else { + return totalBytesRead + " / " + row.taskMetrics.shuffleReadMetrics.recordsRead; + } + } else { + return ""; + } + } else { + return ""; + } + }, + name: "Shuffle Read Size / Records" + }, { data : function (row, type) { if ("taskMetrics" in row) { if (row.taskMetrics.memoryBytesSpilled > 0) { - showMemSpill = true; return type === 'display' ? formatBytes(row.taskMetrics.memoryBytesSpilled, type) : row.taskMetrics.memoryBytesSpilled; } else { return ""; @@ -621,7 +713,6 @@ $(document).ready(function () { data : function (row, type) { if ("taskMetrics" in row) { if (row.taskMetrics.diskBytesSpilled > 0) { - showDiskSpill = true; return type === 'display' ? formatBytes(row.taskMetrics.diskBytesSpilled, type) : row.taskMetrics.diskBytesSpilled; } else { return ""; @@ -655,13 +746,7 @@ $(document).ready(function () { { "visible": false, "targets": 15 }, { "visible": false, "targets": 16 }, { "visible": false, "targets": 17 }, - { "visible": false, "targets": 18 }, - { "visible": false, "targets": 19 }, - { "visible": false, "targets": 20 }, - { "visible": false, "targets": 21 }, - { "visible": false, "targets": 22 }, - { "visible": false, "targets": 23 }, - { "visible": false, "targets": 24 } + { "visible": false, "targets": 18 } ], }; var taskTableSelector = $(taskTable).DataTable(task_conf); @@ -716,25 +801,6 @@ $(document).ready(function () { taskTableSelector.column(18).visible(true); $("#accumulator-update-table").show(); } - - if (showInputMetrics) { - taskTableSelector.column(19).visible(true); - } - if (showOutputMetrics) { - taskTableSelector.column(20).visible(true); - } - if (showShuffleWriteTimeMetrics) { - taskTableSelector.column(21).visible(true); - } - if (showShuffleWriteBytesMetrics) { - taskTableSelector.column(22).visible(true); - } - if (showMemSpill) { - taskTableSelector.column(23).visible(true); - } - if (showDiskSpill) { - taskTableSelector.column(24).visible(true); - } }); }); }); diff --git a/core/src/main/resources/org/apache/spark/ui/static/stagespage-template.html b/core/src/main/resources/org/apache/spark/ui/static/stagespage-template.html index 8f2f7a504bbd0..a27fb96d082c8 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/stagespage-template.html +++ b/core/src/main/resources/org/apache/spark/ui/static/stagespage-template.html @@ -110,6 +110,7 @@

Output Size / Records Write Time Shuffle Write Size / Records + Shuffle Read Size / Records Shuffle Spill (Memory) Shuffle Spill (Disk) Errors diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala index b20b212856ad9..bbb8b4492925a 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala @@ -391,6 +391,7 @@ private[spark] class AppStatusStore( ordered.skip(offset).max(length).asScala.map(_.toApi).toSeq } + // Filters task list based on search parameter def filterTaskList( taskDataList: Seq[v1.TaskData], searchValue: String): Seq[v1.TaskData] = { @@ -415,6 +416,8 @@ private[spark] class AppStatusStore( || f.taskMetrics.get.inputMetrics.recordsRead.toString.contains(searchValue) || f.taskMetrics.get.outputMetrics.bytesWritten.toString.contains(searchValue) || f.taskMetrics.get.outputMetrics.recordsWritten.toString.contains(searchValue) + || f.taskMetrics.get.shuffleReadMetrics.fetchWaitTime.toString.contains(searchValue) + || f.taskMetrics.get.shuffleReadMetrics.recordsRead.toString.contains(searchValue) || f.taskMetrics.get.shuffleWriteMetrics.bytesWritten.toString.contains(searchValue) || f.taskMetrics.get.shuffleWriteMetrics.recordsWritten.toString.contains(searchValue) || f.taskMetrics.get.shuffleWriteMetrics.writeTime.toString.contains(searchValue) diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala index 8295c8b97aadf..7ab3adc0c168d 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala @@ -51,7 +51,6 @@ private[v1] class StagesResource extends BaseAppResource { var executorLogs = ui.store.executorSummary(execId).executorLogs var hostPort = ui.store.executorSummary(execId).hostPort var taskDataArray = ret(i).tasks.get.keys.toArray - var executorStageSummaryArray = ret(i).executorSummary.get.keys.toArray ret(i).executorSummary.get.get(execId).get.executorLogs = executorLogs ret(i).executorSummary.get.get(execId).get.hostPort = hostPort for (taskData <- taskDataArray) { @@ -128,18 +127,18 @@ private[v1] class StagesResource extends BaseAppResource { @PathParam("stageAttemptId") stageAttemptId: Int, @QueryParam("details") @DefaultValue("true") details: Boolean, @Context uriInfo: UriInfo): util.HashMap[String, Object] = { withUI { ui => - val abc = uriInfo.getQueryParameters(true) - val totalRecords = abc.getFirst("numTasks") + val uriQueryParameters = uriInfo.getQueryParameters(true) + val totalRecords = uriQueryParameters.getFirst("numTasks") var isSearch = false var searchValue: String = null var _tasksToShow: Seq[TaskData] = null - if (abc.getFirst("search[value]") != null && abc.getFirst("search[value]").length > 0) { + if (uriQueryParameters.getFirst("search[value]") != null && uriQueryParameters.getFirst("search[value]").length > 0) { _tasksToShow = ui.store.taskList(stageId, stageAttemptId, 0, totalRecords.toInt, indexName("Index"), true) isSearch = true - searchValue = abc.getFirst("search[value]") + searchValue = uriQueryParameters.getFirst("search[value]") } else { - _tasksToShow = doPagination(abc, stageId, stageAttemptId) + _tasksToShow = doPagination(uriQueryParameters, stageId, stageAttemptId) } if (_tasksToShow.nonEmpty) { val iterator = _tasksToShow.iterator @@ -151,13 +150,14 @@ private[v1] class StagesResource extends BaseAppResource { t1.schedulerDelay = AppStatusUtils.schedulerDelay(t1) t1.gettingResultTime = AppStatusUtils.gettingResultTime(t1) } - val ret5 = new util.HashMap[String, Object]() + val ret = new util.HashMap[String, Object]() + // Performs server-side search based on input from user if (isSearch) { val filteredTaskList = ui.store.filterTaskList(_tasksToShow, searchValue) if (filteredTaskList.length > 0) { - ret5.put("aaData", filteredTaskList) + ret.put("aaData", filteredTaskList) } else { - _tasksToShow = doPagination(abc, stageId, stageAttemptId) + _tasksToShow = doPagination(uriQueryParameters, stageId, stageAttemptId) val iterator = _tasksToShow.iterator while(iterator.hasNext) { val t1: TaskData = iterator.next() @@ -167,20 +167,21 @@ private[v1] class StagesResource extends BaseAppResource { t1.schedulerDelay = AppStatusUtils.schedulerDelay(t1) t1.gettingResultTime = AppStatusUtils.gettingResultTime(t1) } - ret5.put("aaData", _tasksToShow) + ret.put("aaData", _tasksToShow) } } else { - ret5.put("aaData", _tasksToShow) + ret.put("aaData", _tasksToShow) } - ret5.put("recordsTotal", totalRecords) - ret5.put("recordsFiltered", totalRecords) - ret5 + ret.put("recordsTotal", totalRecords) + ret.put("recordsFiltered", totalRecords) + ret } else { throw new NotFoundException(s"unknown stage: $stageId") } } } + // Performs pagination on the server side def doPagination(queryParameters: MultivaluedMap[String, String], stageId: Int, stageAttemptId: Int): Seq[TaskData] = { val queryParams = queryParameters.keySet() var columnToSort = 0 From 6a0622e94c3a866ec2c3c9e1177e49a11dd7d9c3 Mon Sep 17 00:00:00 2001 From: pgandhi Date: Sun, 1 Jul 2018 14:43:12 -0500 Subject: [PATCH 10/42] [SPARK-21809] : Fixing scalastyle tests --- .../org/apache/spark/status/AppStatusStore.scala | 3 ++- .../spark/status/api/v1/StagesResource.scala | 15 ++++++++++----- 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala index bbb8b4492925a..c7568fc64041f 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala @@ -421,7 +421,8 @@ private[spark] class AppStatusStore( || f.taskMetrics.get.shuffleWriteMetrics.bytesWritten.toString.contains(searchValue) || f.taskMetrics.get.shuffleWriteMetrics.recordsWritten.toString.contains(searchValue) || f.taskMetrics.get.shuffleWriteMetrics.writeTime.toString.contains(searchValue) - || f.schedulerDelay.toString.contains(searchValue) || f.gettingResultTime.toString.contains(searchValue))) + || f.schedulerDelay.toString.contains(searchValue) + || f.gettingResultTime.toString.contains(searchValue))) filteredTaskDataSequence } diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala index 7ab3adc0c168d..18f91259a16b5 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala @@ -55,8 +55,10 @@ private[v1] class StagesResource extends BaseAppResource { ret(i).executorSummary.get.get(execId).get.hostPort = hostPort for (taskData <- taskDataArray) { ret(i).tasks.get.get(taskData).get.executorLogs = executorLogs - ret(i).tasks.get.get(taskData).get.schedulerDelay = AppStatusUtils.schedulerDelay(ret(i).tasks.get.get(taskData).get) - ret(i).tasks.get.get(taskData).get.gettingResultTime = AppStatusUtils.gettingResultTime(ret(i).tasks.get.get(taskData).get) + ret(i).tasks.get.get(taskData).get.schedulerDelay = + AppStatusUtils.schedulerDelay(ret(i).tasks.get.get(taskData).get) + ret(i).tasks.get.get(taskData).get.gettingResultTime = + AppStatusUtils.gettingResultTime(ret(i).tasks.get.get(taskData).get) } } } @@ -125,14 +127,16 @@ private[v1] class StagesResource extends BaseAppResource { def taskTable( @PathParam("stageId") stageId: Int, @PathParam("stageAttemptId") stageAttemptId: Int, - @QueryParam("details") @DefaultValue("true") details: Boolean, @Context uriInfo: UriInfo): util.HashMap[String, Object] = { + @QueryParam("details") @DefaultValue("true") details: Boolean, @Context uriInfo: UriInfo): + util.HashMap[String, Object] = { withUI { ui => val uriQueryParameters = uriInfo.getQueryParameters(true) val totalRecords = uriQueryParameters.getFirst("numTasks") var isSearch = false var searchValue: String = null var _tasksToShow: Seq[TaskData] = null - if (uriQueryParameters.getFirst("search[value]") != null && uriQueryParameters.getFirst("search[value]").length > 0) { + if (uriQueryParameters.getFirst("search[value]") != null && + uriQueryParameters.getFirst("search[value]").length > 0) { _tasksToShow = ui.store.taskList(stageId, stageAttemptId, 0, totalRecords.toInt, indexName("Index"), true) isSearch = true @@ -182,7 +186,8 @@ private[v1] class StagesResource extends BaseAppResource { } // Performs pagination on the server side - def doPagination(queryParameters: MultivaluedMap[String, String], stageId: Int, stageAttemptId: Int): Seq[TaskData] = { + def doPagination(queryParameters: MultivaluedMap[String, String], stageId: Int, stageAttemptId: Int): + Seq[TaskData] = { val queryParams = queryParameters.keySet() var columnToSort = 0 if (queryParams.contains("order[0][column]")) { From 132fce983a3c23798e9141e36b3ab06a42814a24 Mon Sep 17 00:00:00 2001 From: pgandhi Date: Sun, 1 Jul 2018 14:54:24 -0500 Subject: [PATCH 11/42] [SPARK-21809] : Fixing Scalastyle test again --- .../scala/org/apache/spark/status/api/v1/StagesResource.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala index 18f91259a16b5..53ffc23e74ae1 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala @@ -186,8 +186,8 @@ private[v1] class StagesResource extends BaseAppResource { } // Performs pagination on the server side - def doPagination(queryParameters: MultivaluedMap[String, String], stageId: Int, stageAttemptId: Int): - Seq[TaskData] = { + def doPagination(queryParameters: MultivaluedMap[String, String], stageId: Int, + stageAttemptId: Int): Seq[TaskData] = { val queryParams = queryParameters.keySet() var columnToSort = 0 if (queryParams.contains("order[0][column]")) { From 573390d0a933e7a2641f944046442a136bba6cd8 Mon Sep 17 00:00:00 2001 From: pgandhi Date: Mon, 10 Sep 2018 17:48:36 -0400 Subject: [PATCH 12/42] [SPARK-21809] : Changing comment to point to generic url --- core/src/main/resources/org/apache/spark/ui/static/stagepage.js | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js index f515dbc773718..30c50e7354234 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js @@ -37,7 +37,7 @@ $.extend( $.fn.dataTable.ext.type.order, { } ); // This function will only parse the URL under certain format -// e.g. https://axonitered-jt1.red.ygrid.yahoo.com:50509/history/application_1502220952225_59143/stages/stage/?id=0&attempt=0 +// e.g. https://domain:50509/history/application_1502220952225_59143/stages/stage/?id=0&attempt=0 function stageEndPoint(appId) { var words = document.baseURI.split('/'); var words2 = document.baseURI.split('?'); From af83b6e300a009cea48915fb8471e3edfc4db66b Mon Sep 17 00:00:00 2001 From: pgandhi Date: Fri, 14 Sep 2018 13:52:12 -0400 Subject: [PATCH 13/42] [SPARK-21809] : Addressing Reviews September 14, 2018 --- .../org/apache/spark/ui/static/stagepage.js | 553 +++++++++++------- .../org/apache/spark/ui/static/utils.js | 4 + .../org/apache/spark/ui/static/webui.css | 4 + .../spark/status/api/v1/StagesResource.scala | 26 +- 4 files changed, 359 insertions(+), 228 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js index 30c50e7354234..3038a2073d0fa 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js @@ -68,46 +68,62 @@ function stageEndPoint(appId) { } function getColumnNameForTaskMetricSummary(columnKey) { - if(columnKey == "executorRunTime") { - return "Duration"; - } - else if(columnKey == "jvmGcTime") { - return "GC Time"; - } - else if(columnKey == "gettingResultTime") { - return "Getting Result Time"; - } - else if(columnKey == "inputMetrics") { - return "Input Size / Records"; - } - else if(columnKey == "outputMetrics") { - return "Output Size / Records"; - } - else if(columnKey == "peakExecutionMemory") { - return "Peak Execution Memory"; - } - else if(columnKey == "resultSerializationTime") { - return "Result Serialization Time"; - } - else if(columnKey == "schedulerDelay") { - return "Scheduler Delay"; - } - else if(columnKey == "diskBytesSpilled") { - return "Shuffle spill (disk)"; - } - else if(columnKey == "memoryBytesSpilled") { - return "Shuffle spill (memory)"; - } - else if(columnKey == "shuffleReadMetrics") { - return "Shuffle Read Size / Records"; - } - else if(columnKey == "shuffleWriteMetrics") { - return "Shuffle Write Size / Records"; - } - else if(columnKey == "executorDeserializeTime") { - return "Task Deserialization Time"; + switch(columnKey) { + case "executorRunTime": + return "Duration"; + break; + + case "jvmGcTime": + return "GC Time"; + break; + + case "gettingResultTime": + return "Getting Result Time"; + break; + + case "inputMetrics": + return "Input Size / Records"; + break; + + case "outputMetrics": + return "Output Size / Records"; + break; + + case "peakExecutionMemory": + return "Peak Execution Memory"; + break; + + case "resultSerializationTime": + return "Result Serialization Time"; + break; + + case "schedulerDelay": + return "Scheduler Delay"; + break; + + case "diskBytesSpilled": + return "Shuffle spill (disk)"; + break; + + case "memoryBytesSpilled": + return "Shuffle spill (memory)"; + break; + + case "shuffleReadMetrics": + return "Shuffle Read Size / Records"; + break; + + case "shuffleWriteMetrics": + return "Shuffle Write Size / Records"; + break; + + case "executorDeserializeTime": + return "Task Deserialization Time"; + break; + + default: + return "NA"; } - return "NA"; } $(document).ready(function () { @@ -118,37 +134,71 @@ $(document).ready(function () { "" + " Show Additional Metrics" + "" + - "