From 01c9244d1a2d4aac69cd7b0d2535ce65ae62ce08 Mon Sep 17 00:00:00 2001 From: pgandhi Date: Tue, 12 Sep 2017 09:06:22 -0500 Subject: [PATCH 01/10] [SPARK-21809] : Change Stage Page to use datatables to support sorting columns and searching Converted static html tables to datatables on stage page [SPARK-21809] : Change Stage Page to use datatables to support sorting columns and searching --- .../ui/static/executorspage-template.html | 4 +- .../apache/spark/ui/static/executorspage.js | 37 +- .../spark/ui/static/stagespage-template.html | 112 +++++ .../org/apache/spark/ui/static/taskspages.js | 446 ++++++++++++++++++ .../org/apache/spark/ui/static/utils.js | 61 +++ .../org/apache/spark/ui/static/webui.css | 11 +- .../status/api/v1/AllStagesResource.scala | 74 ++- .../status/api/v1/OneStageResource.scala | 9 +- .../org/apache/spark/status/api/v1/api.scala | 17 +- .../apache/spark/ui/exec/ExecutorsTab.scala | 15 +- .../apache/spark/ui/jobs/ExecutorTable.scala | 175 ------- .../org/apache/spark/ui/jobs/StagePage.scala | 343 +------------- .../org/apache/spark/ui/jobs/UIData.scala | 1 + .../api/v1/AllStagesResourceSuite.scala | 3 +- 14 files changed, 744 insertions(+), 564 deletions(-) create mode 100644 core/src/main/resources/org/apache/spark/ui/static/stagespage-template.html create mode 100644 core/src/main/resources/org/apache/spark/ui/static/taskspages.js 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/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/taskspages.js b/core/src/main/resources/org/apache/spark/ui/static/taskspages.js new file mode 100644 index 0000000000000..507686a405fc5 --- /dev/null +++ b/core/src/main/resources/org/apache/spark/ui/static/taskspages.js @@ -0,0 +1,446 @@ +/* + * 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. + */ + +$(document).ajaxStop($.unblockUI); +$(document).ajaxStart(function () { + $.blockUI({message: '

Loading Tasks Page...

'}); +}); + +$.extend( $.fn.dataTable.ext.type.order, { + "file-size-pre": ConvertDurationString, + + "file-size-asc": function ( a, b ) { + a = ConvertDurationString( a ); + b = ConvertDurationString( b ); + return ((a < b) ? -1 : ((a > b) ? 1 : 0)); + }, + + "file-size-desc": function ( a, b ) { + a = ConvertDurationString( a ); + b = ConvertDurationString( b ); + return ((a < b) ? 1 : ((a > b) ? -1 : 0)); + } +} ); + +function createTemplateURI(appId) { + var words = document.baseURI.split('/'); + var ind = words.indexOf("proxy"); + if (ind > 0) { + var baseURI = words.slice(0, ind + 1).join('/') + '/' + appId + '/static/stagespage-template.html'; + return baseURI; + } + ind = words.indexOf("history"); + if(ind > 0) { + var baseURI = words.slice(0, ind).join('/') + '/static/stagespage-template.html'; + return baseURI; + } + return location.origin + "/static/stagespage-template.html"; +} + +// This function will only parse the URL under certain formate +// e.g. https://axonitered-jt1.red.ygrid.yahoo.com:50509/history/application_1502220952225_59143/stages/stage/?id=0&attempt=0 +function StageEndPoint(appId) { + var words = document.baseURI.split('/'); + var ind = words.indexOf("proxy"); + if (ind > 0) { + var appId = words[ind + 1]; + var stageIdLen = words[ind + 3].indexOf('&'); + var stageId = words[ind + 3].substr(9, stageIdLen - 9); + var newBaseURI = words.slice(0, ind + 2).join('/'); + return newBaseURI + "/api/v1/applications/" + appId + "/stages/" + stageId; + } + ind = words.indexOf("history"); + if (ind > 0) { + var appId = words[ind + 1]; + var attemptId = words[ind + 4].split("&attempt=").pop(); + var stageIdLen = words[ind + 4].indexOf('&'); + var stageId = words[ind + 4].substr(4, stageIdLen - 4); + var newBaseURI = words.slice(0, ind).join('/'); + if (isNaN(attemptId) || attemptId == "0") { + return newBaseURI + "/api/v1/applications/" + appId + "/stages/" + stageId; + } else { + return newBaseURI + "/api/v1/applications/" + appId + "/" + attemptId + "/stages/" + stageId; + } + } + var stageIdLen = words[ind + 3].indexOf('&'); + var stageId = words[ind + 3].substr(9, stageIdLen - 9); + return location.origin + "/api/v1/applications/" + appId + "/stages/" + stageId; +} + +function sortNumber(a,b) { + return a - b; +} + +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]; + } + return result; +} + +$(document).ready(function () { + $.extend($.fn.dataTable.defaults, { + stateSave: true, + lengthMenu: [[20, 40, 60, 100, -1], [20, 40, 60, 100, "All"]], + pageLength: 20 + }); + + $("#showAdditionalMetrics").append( + "
" + + "" + + " Show Additional Metrics" + + "
" + + ""); + + tasksSummary = $("#active-tasks"); + getStandAloneAppId(function (appId) { + + 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[ix]); + }); + + // prepare data for task aggregated metrics table + indices = Object.keys(response[0].executorSummary); + var task_summary_table = []; + indices.forEach(function (ix){ + response[0].executorSummary[ix].id = ix; + 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 = []; + + task_table.forEach(function (x){ + durationSummary.push(x.taskMetrics.executorRunTime); + 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 = []; + 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); + + 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); + } + + // prepare data for accumulatorUpdates + var indices = Object.keys(response[0].accumulatorUpdates); + var accumulator_table_all = []; + var accumulator_table = []; + indices.forEach(function (ix){ + accumulator_table_all.push(response[0].accumulatorUpdates[ix]); + }); + + accumulator_table_all.forEach(function (x){ + var name = (x.name).toString(); + if(name.includes("internal.") == false){ + accumulator_table.push(x); + } + }); + + // rendering the UI page + var data = {executors: response, "taskstable": task_table, "task_metrics_table": task_metrics_table}; + $.get(createTemplateURI(appId), function(template) { + tasksSummary.append(Mustache.render($(template).filter("#stages-summary-template").html(), data)); + + $("#taskMetric").click(function(){ + $("#arrowtoggle1").toggleClass("arrow-open arrow-closed"); + $("#toggle-metrics").toggle(); + }); + + $("#aggregatedMetrics").click(function(){ + $("#arrowtoggle2").toggleClass("arrow-open arrow-closed"); + $("#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) { + return row.metric != 'Peak Execution Memory' ? (formatDuration(row.p0)) : formatBytes(row.p0, type); + } + }, + { + data: function (row, type) { + return row.metric != 'Peak Execution Memory' ? (formatDuration(row.p25)) : formatBytes(row.p25, type); + } + }, + { + data: function (row, type) { + return row.metric != 'Peak Execution Memory' ? (formatDuration(row.p50)) : formatBytes(row.p50, type); + } + }, + { + data: function (row, type) { + return row.metric != 'Peak Execution Memory' ? (formatDuration(row.p75)) : formatBytes(row.p75, type); + } + }, + { + data: function (row, type) { + return row.metric != 'Peak Execution Memory' ? (formatDuration(row.p100)) : formatBytes(row.p100, type); + } + } + ], + "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 + var tasksSummarytable = "#summary-stages-table"; + var task_summary_conf = { + "data": task_summary_table, + "columns": [ + {data : "id"}, + {data : "executorLogs", render: formatLogsCells}, + {data : "host"}, + { + data : function (row, type) { + return type === 'display' ? formatDuration(row.taskTime) : row.taskTime; + } + }, + { + data : function (row, type) { + var totaltasks = row.succeededTasks + row.failedTasks + row.killedTasks; + return type === 'display' ? totaltasks : totaltasks.toString(); + } + }, + {data : "failedTasks"}, + {data : "killedTasks"}, + {data : "succeededTasks"}, + {data : "blacklisted"}, + { + data : function (row, type) { + return row.inputRecords != 0 ? formatBytes(row.inputBytes/row.inputRecords) : 0; + } + }, + { + data : function (row, type) { + return row.outputRecords != 0 ? formatBytes(row.outputBytes/row.outputRecords) : 0; + } + }, + { + data : function (row, type) { + return row.shuffleReadRecords != 0 ? formatBytes(row.shuffleRead/row.shuffleReadRecords) : 0; + } + }, + { + data : function (row, type) { + return row.shuffleWriteRecords != 0 ? formatBytes(row.shuffleWrite/row.shuffleWriteRecords) : 0; + } + }, + { + data : function (row, type) { + return typeof row.memoryBytesSpilled != 'undefined' ? formatBytes(row.memoryBytesSpilled) : ""; + } + }, + { + data : function (row, type) { + return typeof row.diskBytesSpilled != 'undefined' ? formatBytes(row.diskBytesSpilled) : ""; + } + } + ], + "order": [[0, "asc"]] + } + $(tasksSummarytable).DataTable(task_summary_conf); + $('#active-tasks [data-toggle="tooltip"]').tooltip(); + + // building accumulator update table + var accumulatorTable = "#accumulator-table"; + var accumulator_conf = { + "data": accumulator_table, + "columns": [ + {data : "id"}, + {data : "name"}, + {data : "value"} + ], + "paging": false, + "searching": false, + "order": [[0, "asc"]] + } + $(accumulatorTable).DataTable(accumulator_conf); + + // building tasks table + var taskTable = "#active-tasks-table"; + var task_conf = { + "data": task_table, + "columns": [ + {data: function (row, type) { + return type !== 'display' ? (isNaN(row.index) ? 0 : row.index ) : row.index; + } + }, + {data : "taskId"}, + {data : "attempt"}, + {data : "taskState"}, + {data : "taskLocality"}, + { + data : function (row, type) { + return row.executorId + ' / ' + row.host; + } + }, + {data : "executorLogs", render: formatLogsCells}, + {data : "launchTime"}, + { + data : function (row, type) { + return type === 'display' ? formatDuration(row.taskMetrics.executorRunTime) : row.taskMetrics.executorRunTime; + } + }, + { + data : function (row, type) { + return type === 'display' ? formatDuration(row.taskMetrics.jvmGcTime) : row.taskMetrics.jvmGcTime; + } + }, + { + data : function (row, type) { + return type === 'display' ? formatDuration(row.schedulerDelay) : row.schedulerDelay; + } + }, + { + data : function (row, type) { + return type === 'display' ? formatDuration(row.taskMetrics.executorDeserializeTime) : row.taskMetrics.executorDeserializeTime; + } + }, + { + data : function (row, type) { + return type === 'display' ? formatDuration(row.taskMetrics.resultSerializationTime) : row.taskMetrics.resultSerializationTime; + } + }, + { + data : function (row, type) { + return type === 'display' ? formatDuration(row.gettingResultTime) : row.gettingResultTime; + } + }, + { + data : function (row, type) { + return type === 'display' ? formatDuration(row.taskMetrics.peakExecutionMemory) : row.taskMetrics.peakExecutionMemory; + } + }, + { + data : function (row, type) { + var msg = row.errorMessage; + if (typeof msg === 'undefined'){ + return ""; + } else { + var form_head = msg.substring(0, msg.indexOf("at")); + var form = "+details"; + var form_msg = "
" + row.errorMessage + "
"; + return form_head + form + form_msg; + } + } + } + ], + "columnDefs": [ + { "visible": false, "targets": 10 }, + { "visible": false, "targets": 11 }, + { "visible": false, "targets": 12 }, + { "visible": false, "targets": 13 }, + { "visible": false, "targets": 14 } + ], + "order": [[0, "asc"]] + }; + var taskTableSelector = $(taskTable).DataTable(task_conf); + + // hide or show columns dynamically event + $('input.toggle-vis').on('click', function(e){ + // Get the column + var para = $(this).attr('data-column'); + if(para == "0"){ + var column = taskTableSelector.column([10, 11, 12, 13, 14]); + if($(this).is(":checked")){ + $(".toggle-vis").prop('checked', true); + column.visible(true); + } else { + $(".toggle-vis").prop('checked', false); + column.visible(false); + } + } else { + var column = taskTableSelector.column($(this).attr('data-column')); + // Toggle the visibility + column.visible(!column.visible()); + } + }); + + // title number and toggle list + $("#summaryMetricsTitle").html("Summary Metrics for " + task_table.length + " Completed Tasks"); + $("#tasksTitle").html("Task (" + task_table.length + ")"); + + // hide or show the accumulate update table + if(accumulator_table.length == 0){ + $("accumulator-update-table").hide(); + } + }); + }); + }); +}); 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 edc0ee2ce181d..8431c0dca8223 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 @@ -46,3 +46,64 @@ function formatBytes(bytes, type) { var i = Math.floor(Math.log(bytes) / Math.log(k)); return parseFloat((bytes / Math.pow(k, i)).toFixed(dm)) + ' ' + sizes[i]; } + +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; +} 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..5ed5886a7b703 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; -} \ No newline at end of file +} + +.title-table { + clear: left; + display: inline-block; +} + +.table-dataTable{ + width="100%"; +} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala index 4a4ed954d689e..ec0f5a92a9346 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala @@ -20,7 +20,8 @@ import java.util.{Arrays, Date, List => JList} import javax.ws.rs.{GET, Produces, QueryParam} import javax.ws.rs.core.MediaType -import org.apache.spark.scheduler.{AccumulableInfo => InternalAccumulableInfo, StageInfo} +import org.apache.spark.scheduler.{AccumulableInfo => InternalAccumulableInfo, StageInfo, TaskInfo} +import org.apache.spark.ui.exec.ExecutorsListener import org.apache.spark.ui.SparkUI import org.apache.spark.ui.jobs.UIData.{StageUIData, TaskUIData} import org.apache.spark.ui.jobs.UIData.{InputMetricsUIData => InternalInputMetrics, OutputMetricsUIData => InternalOutputMetrics, ShuffleReadMetricsUIData => InternalShuffleReadMetrics, ShuffleWriteMetricsUIData => InternalShuffleWriteMetrics, TaskMetricsUIData => InternalTaskMetrics} @@ -48,7 +49,8 @@ private[v1] class AllStagesResource(ui: SparkUI) { } } yield { stageUiData.lastUpdateTime = ui.lastUpdateTime - AllStagesResource.stageUiToStageData(status, stageInfo, stageUiData, includeDetails = false) + AllStagesResource.stageUiToStageData( + status, stageInfo, stageUiData, ui.executorsListener, includeDetails = false) } } } @@ -58,6 +60,7 @@ private[v1] object AllStagesResource { status: StageStatus, stageInfo: StageInfo, stageUiData: StageUIData, + executorLog: ExecutorsListener = null, includeDetails: Boolean): StageData = { val taskLaunchTimes = stageUiData.taskData.values.map(_.taskInfo.launchTime).filter(_ > 0) @@ -70,23 +73,37 @@ private[v1] object AllStagesResource { } val taskData = if (includeDetails) { - Some(stageUiData.taskData.map { case (k, v) => - k -> convertTaskData(v, stageUiData.lastUpdateTime) }) + Some(stageUiData.taskData.map { case (k, v) => k -> convertTaskData(v, + stageUiData.lastUpdateTime, executorLog) } ) } else { None } + val executorSummary = if (includeDetails) { Some(stageUiData.executorSummary.map { case (k, summary) => k -> new ExecutorStageSummary( taskTime = summary.taskTime, failedTasks = summary.failedTasks, succeededTasks = summary.succeededTasks, + killedTasks = summary.killedTasks, inputBytes = summary.inputBytes, + inputRecords = summary.inputRecords, outputBytes = summary.outputBytes, + outputRecords = summary.outputRecords, shuffleRead = summary.shuffleRead, + shuffleReadRecords = summary.shuffleReadRecords, shuffleWrite = summary.shuffleWrite, + shuffleWriteRecords = summary.shuffleWriteRecords, memoryBytesSpilled = summary.memoryBytesSpilled, - diskBytesSpilled = summary.diskBytesSpilled + diskBytesSpilled = summary.diskBytesSpilled, + blacklisted = summary.isBlacklisted, + host = executorLog.getExecutorHost(k.toString), + executorLogs = if (executorLog != null) { + executorLog.executorToTaskSummary.get(k.toString) + .map(_.executorLogs).getOrElse(Map.empty) + } else { + Map.empty + } ) }) } else { @@ -138,21 +155,61 @@ private[v1] object AllStagesResource { } } - def convertTaskData(uiData: TaskUIData, lastUpdateTime: Option[Long]): TaskData = { + private def getGettingResultTime(info: TaskInfo, currentTime: Long): Long = { + if (info.gettingResult) { + if (info.finished) { + info.finishTime - info.gettingResultTime + } else { + // The task is still fetching the result. + System.currentTimeMillis() - info.gettingResultTime + } + } else { + 0L + } + } + + private def getSchedulerDelay( + info: TaskInfo, metrics: InternalTaskMetrics, currentTime: Long): Long = { + if (info.finished) { + val totalExecutionTime = info.finishTime - info.launchTime + val executorOverhead = metrics.executorDeserializeTime + + metrics.resultSerializationTime + math.max( + 0, + totalExecutionTime - metrics.executorRunTime - executorOverhead - + getGettingResultTime(info, currentTime)) + } else { + // The task is still running and the metrics like executorRunTime are not available. + 0L + } + } + + def convertTaskData(uiData: TaskUIData, lastUpdateTime: Option[Long], + executorLog: ExecutorsListener): TaskData = { + var currentTime = System.currentTimeMillis() new TaskData( taskId = uiData.taskInfo.taskId, index = uiData.taskInfo.index, attempt = uiData.taskInfo.attemptNumber, launchTime = new Date(uiData.taskInfo.launchTime), - duration = uiData.taskDuration(lastUpdateTime), + duration = uiData.taskDuration(lastUpdateTime), + gettingResultTime = getGettingResultTime(uiData.taskInfo, currentTime), + schedulerDelay = getSchedulerDelay(uiData.taskInfo, uiData.metrics.get, currentTime), executorId = uiData.taskInfo.executorId, host = uiData.taskInfo.host, status = uiData.taskInfo.status, taskLocality = uiData.taskInfo.taskLocality.toString(), + taskState = uiData.taskInfo.status.toString, speculative = uiData.taskInfo.speculative, accumulatorUpdates = uiData.taskInfo.accumulables.map { convertAccumulableInfo }, errorMessage = uiData.errorMessage, - taskMetrics = uiData.metrics.map { convertUiTaskMetrics } + taskMetrics = uiData.metrics.map { convertUiTaskMetrics }, + executorLogs = if (executorLog != null) { + executorLog.executorToTaskSummary.get(uiData.taskInfo.executorId) + .map(_.executorLogs).getOrElse(Map.empty) + } else { + Map.empty + } ) } @@ -252,6 +309,7 @@ private[v1] object AllStagesResource { executorDeserializeCpuTime = internal.executorDeserializeCpuTime, executorRunTime = internal.executorRunTime, executorCpuTime = internal.executorCpuTime, + peakExecutionMemory = internal.peakExecutionMemory, resultSize = internal.resultSize, jvmGcTime = internal.jvmGCTime, resultSerializationTime = internal.resultSerializationTime, diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneStageResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/OneStageResource.scala index f15073bccced2..479da4d36b2f5 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/OneStageResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/OneStageResource.scala @@ -37,7 +37,7 @@ private[v1] class OneStageResource(ui: SparkUI) { stageAttempts.map { stage => stage.ui.lastUpdateTime = ui.lastUpdateTime AllStagesResource.stageUiToStageData(stage.status, stage.info, stage.ui, - includeDetails = true) + ui.executorsListener, includeDetails = true) } } } @@ -50,7 +50,7 @@ private[v1] class OneStageResource(ui: SparkUI) { withStageAttempt(stageId, stageAttemptId) { stage => stage.ui.lastUpdateTime = ui.lastUpdateTime AllStagesResource.stageUiToStageData(stage.status, stage.info, stage.ui, - includeDetails = true) + ui.executorsListener, includeDetails = true) } } @@ -83,8 +83,9 @@ private[v1] class OneStageResource(ui: SparkUI) { @DefaultValue("20") @QueryParam("length") length: Int, @DefaultValue("ID") @QueryParam("sortBy") sortBy: TaskSorting): Seq[TaskData] = { withStageAttempt(stageId, stageAttemptId) { stage => - val tasks = stage.ui.taskData.values - .map{ AllStagesResource.convertTaskData(_, ui.lastUpdateTime)}.toIndexedSeq + val tasks = stage.ui.taskData.values.map{ + k => AllStagesResource.convertTaskData(k, ui.lastUpdateTime, ui.executorsListener)}. + toIndexedSeq .sorted(OneStageResource.ordering(sortBy)) tasks.slice(offset, offset + length) } 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 05948f2661056..f086ef9d61bc2 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 @@ -49,12 +49,20 @@ class ExecutorStageSummary private[spark]( val taskTime : Long, val failedTasks : Int, val succeededTasks : Int, + val killedTasks: Int, + val blacklisted: Int, val inputBytes : Long, + val inputRecords : Long, val outputBytes : Long, + val outputRecords : Long, val shuffleRead : Long, + val shuffleReadRecords : Long, val shuffleWrite : Long, + val shuffleWriteRecords : Long, val memoryBytesSpilled : Long, - val diskBytesSpilled : Long) + val diskBytesSpilled : Long, + val host : String, + val executorLogs: Map[String, String]) class ExecutorSummary private[spark]( val id: String, @@ -171,20 +179,25 @@ class TaskData private[spark]( val attempt: Int, val launchTime: Date, val duration: Option[Long] = None, + val gettingResultTime: Double, + val schedulerDelay: Double, val executorId: String, val host: String, val status: String, val taskLocality: String, + val taskState: String, val speculative: Boolean, val accumulatorUpdates: Seq[AccumulableInfo], val errorMessage: Option[String] = None, - val taskMetrics: Option[TaskMetrics] = None) + val taskMetrics: Option[TaskMetrics] = None, + val executorLogs: Map[String, String]) class TaskMetrics private[spark]( val executorDeserializeTime: Long, val executorDeserializeCpuTime: Long, val executorRunTime: Long, val executorCpuTime: Long, + val peakExecutionMemory: Long, val resultSize: Long, val jvmGcTime: Long, val resultSerializationTime: Long, diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala index 64a1a292a3840..ddb9b7c40ff3e 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala @@ -17,6 +17,7 @@ package org.apache.spark.ui.exec +import scala.collection.mutable import scala.collection.mutable.{LinkedHashMap, ListBuffer} import org.apache.spark.{Resubmitted, SparkConf, SparkContext} @@ -67,14 +68,13 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener, conf: Spar extends SparkListener { val executorToTaskSummary = LinkedHashMap[String, ExecutorTaskSummary]() var executorEvents = new ListBuffer[SparkListenerEvent]() + val executorIdToAddress = mutable.HashMap[String, String]() private val maxTimelineExecutors = conf.getInt("spark.ui.timeline.executors.maximum", 1000) private val retainedDeadExecutors = conf.getInt("spark.ui.retainedDeadExecutors", 100) def activeStorageStatusList: Seq[StorageStatus] = storageStatusListener.storageStatusList - def deadStorageStatusList: Seq[StorageStatus] = storageStatusListener.deadStorageStatusList - override def onExecutorAdded( executorAdded: SparkListenerExecutorAdded): Unit = synchronized { val eid = executorAdded.executorId @@ -170,6 +170,17 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener, conf: Spar execTaskSummary.isBlacklisted = isBlacklisted } + def getExecutorHost(eid: String): String = { + val host = activeStorageStatusList.find { id => + id.blockManagerId.executorId == eid + } + if( host.nonEmpty ) { + return host.head.blockManagerId.hostPort + } else { + return "CANNOT FIND ADDRESS" + } + } + override def onExecutorBlacklisted( executorBlacklisted: SparkListenerExecutorBlacklisted) : Unit = synchronized { 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 382a6f979f2e6..0000000000000 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ /dev/null @@ -1,175 +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.collection.mutable -import scala.xml.{Node, Unparsed} - -import org.apache.spark.ui.{ToolTips, UIUtils} -import org.apache.spark.ui.jobs.UIData.StageUIData -import org.apache.spark.util.Utils - -/** Stage summary grouped by executors. */ -private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: StagesTab) { - private val listener = parent.progressListener - - def toNodeSeq: Seq[Node] = { - listener.synchronized { - executorTable() - } - } - - /** Special table which merges two header cells. */ - private def executorTable[T](): Seq[Node] = { - val stageData = listener.stageIdToData.get((stageId, stageAttemptId)) - var hasInput = false - var hasOutput = false - var hasShuffleWrite = false - var hasShuffleRead = false - var hasBytesSpilled = false - stageData.foreach { data => - hasInput = data.hasInput - hasOutput = data.hasOutput - hasShuffleRead = data.hasShuffleRead - hasShuffleWrite = data.hasShuffleWrite - hasBytesSpilled = data.hasBytesSpilled - } - - - - - - - - - - - {if (hasInput) { - - }} - {if (hasOutput) { - - }} - {if (hasShuffleRead) { - - }} - {if (hasShuffleWrite) { - - }} - {if (hasBytesSpilled) { - - - }} - - - - {createExecutorTable()} - -
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() : Seq[Node] = { - // Make an executor-id -> address map - val executorIdToAddress = mutable.HashMap[String, String]() - listener.blockManagerIds.foreach { blockManagerId => - val address = blockManagerId.hostPort - val executorId = blockManagerId.executorId - executorIdToAddress.put(executorId, address) - } - - listener.stageIdToData.get((stageId, stageAttemptId)) match { - case Some(stageData: StageUIData) => - stageData.executorSummary.toSeq.sortBy(_._1).map { case (k, v) => - - -
{k}
-
- { - val logs = parent.executorsListener.executorToTaskSummary.get(k) - .map(_.executorLogs).getOrElse(Map.empty) - logs.map { - case (logName, logUrl) =>
{logName}
- } - } -
- - {executorIdToAddress.getOrElse(k, "CANNOT FIND ADDRESS")} - {UIUtils.formatDuration(v.taskTime)} - {v.failedTasks + v.succeededTasks + v.reasonToNumKilled.values.sum} - {v.failedTasks} - {v.reasonToNumKilled.values.sum} - {v.succeededTasks} - {if (stageData.hasInput) { - - {s"${Utils.bytesToString(v.inputBytes)} / ${v.inputRecords}"} - - }} - {if (stageData.hasOutput) { - - {s"${Utils.bytesToString(v.outputBytes)} / ${v.outputRecords}"} - - }} - {if (stageData.hasShuffleRead) { - - {s"${Utils.bytesToString(v.shuffleRead)} / ${v.shuffleReadRecords}"} - - }} - {if (stageData.hasShuffleWrite) { - - {s"${Utils.bytesToString(v.shuffleWrite)} / ${v.shuffleWriteRecords}"} - - }} - {if (stageData.hasBytesSpilled) { - - {Utils.bytesToString(v.memoryBytesSpilled)} - - - {Utils.bytesToString(v.diskBytesSpilled)} - - }} - {v.isBlacklisted} - - } - case None => - Seq.empty[Node] - } - } -} 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 4d80308eb0a6d..2eca83df33f19 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 @@ -22,17 +22,16 @@ import java.util.Date import javax.servlet.http.HttpServletRequest import scala.collection.mutable.HashSet -import scala.xml.{Elem, Node, Unparsed} +import scala.xml.{Node, Unparsed} import org.apache.commons.lang3.StringEscapeUtils import org.apache.spark.SparkConf -import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo, TaskLocality} import org.apache.spark.ui._ import org.apache.spark.ui.exec.ExecutorsListener import org.apache.spark.ui.jobs.UIData._ -import org.apache.spark.util.{Distribution, Utils} +import org.apache.spark.util.Utils /** Page showing statistics and task list for a given stage */ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { @@ -144,7 +143,6 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { val allAccumulables = progressListener.stageIdToData((stageId, stageAttemptId)).accumulables val externalAccumulables = allAccumulables.values.filter { acc => !acc.internal } val hasAccumulators = externalAccumulables.nonEmpty - val summary =
- val showAdditionalMetrics = -
- - - Show Additional Metrics - - -
- val dagViz = UIUtils.showDagVizForStage( stageId, operationGraphListener.getOperationGraphForStage(stageId)) - val accumulableHeaders: Seq[String] = Seq("Accumulable", "Value") - def accumulableRow(acc: AccumulableInfo): Seq[Node] = { - (acc.name, acc.value) match { - case (Some(name), Some(value)) => {name}{value} - case _ => Seq.empty[Node] - } - } - val accumulableTable = UIUtils.listingTable( - accumulableHeaders, - accumulableRow, - externalAccumulables.toSeq) - val page: Int = { // If the user has changed to a larger page size, then go to page 1 in order to avoid // IndexOutOfBoundsException. @@ -340,261 +259,19 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { // Excludes tasks which failed and have incomplete metrics val validTasks = tasks.filter(t => t.taskInfo.status == "SUCCESS" && t.metrics.isDefined) - val summaryTable: Option[Seq[Node]] = - if (validTasks.isEmpty) { - None - } - else { - def getDistributionQuantiles(data: Seq[Double]): IndexedSeq[Double] = - Distribution(data).get.getQuantiles() - def getFormattedTimeQuantiles(times: Seq[Double]): Seq[Node] = { - getDistributionQuantiles(times).map { millis => - {UIUtils.formatDuration(millis.toLong)} - } - } - def getFormattedSizeQuantiles(data: Seq[Double]): Seq[Elem] = { - getDistributionQuantiles(data).map(d => {Utils.bytesToString(d.toLong)}) - } - - val deserializationTimes = validTasks.map { taskUIData: TaskUIData => - taskUIData.metrics.get.executorDeserializeTime.toDouble - } - val deserializationQuantiles = - - - Task Deserialization Time - - +: getFormattedTimeQuantiles(deserializationTimes) - - val serviceTimes = validTasks.map { taskUIData: TaskUIData => - taskUIData.metrics.get.executorRunTime.toDouble - } - val serviceQuantiles = Duration +: getFormattedTimeQuantiles(serviceTimes) - - val gcTimes = validTasks.map { taskUIData: TaskUIData => - taskUIData.metrics.get.jvmGCTime.toDouble - } - val gcQuantiles = - - GC Time - - +: getFormattedTimeQuantiles(gcTimes) - - val serializationTimes = validTasks.map { taskUIData: TaskUIData => - taskUIData.metrics.get.resultSerializationTime.toDouble - } - val serializationQuantiles = - - - Result Serialization Time - - +: getFormattedTimeQuantiles(serializationTimes) - - val gettingResultTimes = validTasks.map { taskUIData: TaskUIData => - getGettingResultTime(taskUIData.taskInfo, currentTime).toDouble - } - val gettingResultQuantiles = - - - Getting Result Time - - +: - getFormattedTimeQuantiles(gettingResultTimes) - - val peakExecutionMemory = validTasks.map { taskUIData: TaskUIData => - taskUIData.metrics.get.peakExecutionMemory.toDouble - } - val peakExecutionMemoryQuantiles = { - - - Peak Execution Memory - - +: getFormattedSizeQuantiles(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 schedulerDelays = validTasks.map { taskUIData: TaskUIData => - getSchedulerDelay(taskUIData.taskInfo, taskUIData.metrics.get, currentTime).toDouble - } - val schedulerDelayTitle = Scheduler Delay - val schedulerDelayQuantiles = schedulerDelayTitle +: - getFormattedTimeQuantiles(schedulerDelays) - def getFormattedSizeQuantilesWithRecords(data: Seq[Double], records: Seq[Double]) - : Seq[Elem] = { - val recordDist = getDistributionQuantiles(records).iterator - getDistributionQuantiles(data).map(d => - {s"${Utils.bytesToString(d.toLong)} / ${recordDist.next().toLong}"} - ) - } - - val inputSizes = validTasks.map { taskUIData: TaskUIData => - taskUIData.metrics.get.inputMetrics.bytesRead.toDouble - } - - val inputRecords = validTasks.map { taskUIData: TaskUIData => - taskUIData.metrics.get.inputMetrics.recordsRead.toDouble - } - - val inputQuantiles = Input Size / Records +: - getFormattedSizeQuantilesWithRecords(inputSizes, inputRecords) - - val outputSizes = validTasks.map { taskUIData: TaskUIData => - taskUIData.metrics.get.outputMetrics.bytesWritten.toDouble - } - - val outputRecords = validTasks.map { taskUIData: TaskUIData => - taskUIData.metrics.get.outputMetrics.recordsWritten.toDouble - } - - val outputQuantiles = Output Size / Records +: - getFormattedSizeQuantilesWithRecords(outputSizes, outputRecords) - - val shuffleReadBlockedTimes = validTasks.map { taskUIData: TaskUIData => - taskUIData.metrics.get.shuffleReadMetrics.fetchWaitTime.toDouble - } - val shuffleReadBlockedQuantiles = - - - Shuffle Read Blocked Time - - +: - getFormattedTimeQuantiles(shuffleReadBlockedTimes) - - val shuffleReadTotalSizes = validTasks.map { taskUIData: TaskUIData => - taskUIData.metrics.get.shuffleReadMetrics.totalBytesRead.toDouble - } - val shuffleReadTotalRecords = validTasks.map { taskUIData: TaskUIData => - taskUIData.metrics.get.shuffleReadMetrics.recordsRead.toDouble - } - val shuffleReadTotalQuantiles = - - - Shuffle Read Size / Records - - +: - getFormattedSizeQuantilesWithRecords(shuffleReadTotalSizes, shuffleReadTotalRecords) - - val shuffleReadRemoteSizes = validTasks.map { taskUIData: TaskUIData => - taskUIData.metrics.get.shuffleReadMetrics.remoteBytesRead.toDouble - } - val shuffleReadRemoteQuantiles = - - - Shuffle Remote Reads - - +: - getFormattedSizeQuantiles(shuffleReadRemoteSizes) - - val shuffleWriteSizes = validTasks.map { taskUIData: TaskUIData => - taskUIData.metrics.get.shuffleWriteMetrics.bytesWritten.toDouble - } - - val shuffleWriteRecords = validTasks.map { taskUIData: TaskUIData => - taskUIData.metrics.get.shuffleWriteMetrics.recordsWritten.toDouble - } - - val shuffleWriteQuantiles = Shuffle Write Size / Records +: - getFormattedSizeQuantilesWithRecords(shuffleWriteSizes, shuffleWriteRecords) - - val memoryBytesSpilledSizes = validTasks.map { taskUIData: TaskUIData => - taskUIData.metrics.get.memoryBytesSpilled.toDouble - } - val memoryBytesSpilledQuantiles = Shuffle spill (memory) +: - getFormattedSizeQuantiles(memoryBytesSpilledSizes) - - val diskBytesSpilledSizes = validTasks.map { taskUIData: TaskUIData => - taskUIData.metrics.get.diskBytesSpilled.toDouble - } - val diskBytesSpilledQuantiles = Shuffle spill (disk) +: - getFormattedSizeQuantiles(diskBytesSpilledSizes) - - val listings: Seq[Seq[Node]] = Seq( - {serviceQuantiles}, - {schedulerDelayQuantiles}, - - {deserializationQuantiles} - - {gcQuantiles}, - - {serializationQuantiles} - , - {gettingResultQuantiles}, - - {peakExecutionMemoryQuantiles} - , - if (stageData.hasInput) {inputQuantiles} else Nil, - if (stageData.hasOutput) {outputQuantiles} else Nil, - if (stageData.hasShuffleRead) { - - {shuffleReadBlockedQuantiles} - - {shuffleReadTotalQuantiles} - - {shuffleReadRemoteQuantiles} - - } else { - Nil - }, - if (stageData.hasShuffleWrite) {shuffleWriteQuantiles} else Nil, - if (stageData.hasBytesSpilled) {memoryBytesSpilledQuantiles} else Nil, - if (stageData.hasBytesSpilled) {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). - Some(UIUtils.listingTable( - quantileHeaders, - identity[Seq[Node]], - listings, - fixedWidth = true, - id = Some("task-summary-table"), - stripeRowsWithCss = false)) - } - - val executorTable = new ExecutorTable(stageId, stageAttemptId, parent) - - val maybeAccumulableTable: Seq[Node] = - if (hasAccumulators) {

Accumulators

++ accumulableTable } else Seq.empty - - val aggMetrics = - -

- - Aggregated Metrics by Executor -

-
-
- {executorTable.toNodeSeq} -
- val content = summary ++ - dagViz ++ - showAdditionalMetrics ++ - makeTimeline( + dagViz ++
++ + makeTimeline( // Only show the tasks in the table stageData.taskData.values.toSeq.filter(t => taskIdsInPage.contains(t.taskInfo.taskId)), currentTime) ++ -

Summary Metrics for {numCompleted} Completed Tasks

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

Tasks ({totalTasksNumStr})

++ - taskTableHTML ++ jsForScrollingDownToTaskTable - UIUtils.headerSparkPage(stageHeader, content, parent, showVisualization = true) +
+ + +
+ UIUtils.headerSparkPage(stageHeader, content, parent, showVisualization = true, + useDataTables = true) } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala index 5acec0d0f54c9..28b8c0a698d86 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala @@ -34,6 +34,7 @@ private[spark] object UIData { var taskTime : Long = 0 var failedTasks : Int = 0 var succeededTasks : Int = 0 + var killedTasks : Int = 0 var reasonToNumKilled : Map[String, Int] = Map.empty var inputBytes : Long = 0 var inputRecords : Long = 0 diff --git a/core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala b/core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala index 82bd7c4ff6604..a833c40f9788b 100644 --- a/core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala @@ -39,7 +39,8 @@ class AllStagesResourceSuite extends SparkFunSuite { val status = StageStatus.ACTIVE val stageInfo = new StageInfo( 1, 1, "stage 1", 10, Seq.empty, Seq.empty, "details abc") - val stageData = AllStagesResource.stageUiToStageData(status, stageInfo, stageUiData, false) + val stageData = AllStagesResource.stageUiToStageData( + status, stageInfo, stageUiData, null, false) stageData.firstTaskLaunchedTime } From 4b94bc0c25322b26207201a4dd30a17d7d8fca39 Mon Sep 17 00:00:00 2001 From: pgandhi Date: Tue, 12 Sep 2017 10:21:12 -0500 Subject: [PATCH 02/10] [SPARK-21809] : Fixing code to pass ScalaStyle Tests [SPARK-21809] : Fixing code to pass ScalaStyle Check --- .../org/apache/spark/status/api/v1/AllStagesResource.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala index ec0f5a92a9346..05a94ac37f2a8 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala @@ -21,8 +21,8 @@ import javax.ws.rs.{GET, Produces, QueryParam} import javax.ws.rs.core.MediaType import org.apache.spark.scheduler.{AccumulableInfo => InternalAccumulableInfo, StageInfo, TaskInfo} -import org.apache.spark.ui.exec.ExecutorsListener import org.apache.spark.ui.SparkUI +import org.apache.spark.ui.exec.ExecutorsListener import org.apache.spark.ui.jobs.UIData.{StageUIData, TaskUIData} import org.apache.spark.ui.jobs.UIData.{InputMetricsUIData => InternalInputMetrics, OutputMetricsUIData => InternalOutputMetrics, ShuffleReadMetricsUIData => InternalShuffleReadMetrics, ShuffleWriteMetricsUIData => InternalShuffleWriteMetrics, TaskMetricsUIData => InternalTaskMetrics} import org.apache.spark.util.Distribution From c58895357d568b5980a8cda7420536c3d87af4c5 Mon Sep 17 00:00:00 2001 From: pgandhi Date: Wed, 13 Sep 2017 15:25:15 -0500 Subject: [PATCH 03/10] [SPARK-21809] : Adding new fields at the end of constructor definition --- .../scala/org/apache/spark/status/api/v1/api.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) 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 f086ef9d61bc2..b0980255a141e 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 @@ -49,18 +49,18 @@ class ExecutorStageSummary private[spark]( val taskTime : Long, val failedTasks : Int, val succeededTasks : Int, - val killedTasks: Int, - val blacklisted: Int, val inputBytes : Long, - val inputRecords : Long, val outputBytes : Long, - val outputRecords : Long, val shuffleRead : Long, - val shuffleReadRecords : Long, val shuffleWrite : Long, val shuffleWriteRecords : Long, val memoryBytesSpilled : Long, val diskBytesSpilled : Long, + val killedTasks: Int, + val blacklisted: Int, + val inputRecords : Long, + val outputRecords : Long, + val shuffleReadRecords : Long, val host : String, val executorLogs: Map[String, String]) From dd12be7db42a4de134d12dfd98dacddb074af4b8 Mon Sep 17 00:00:00 2001 From: pgandhi Date: Wed, 20 Sep 2017 09:05:53 -0500 Subject: [PATCH 04/10] [SPARK-21809]: Adding Problem Filter to MiMa Excludes Adding Problem filter for ExecutorStageSummary to MiMa Excludes --- project/MimaExcludes.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index dd299e074535e..1532a58a8eeaa 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -36,6 +36,9 @@ object MimaExcludes { // Exclude rules for 2.3.x lazy val v23excludes = v22excludes ++ Seq( + // [SPARK-21809] Change Stage Page to use datatables to support sorting columns and searching + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.ExecutorStageSummary.this"), + // [SPARK-20495][SQL] Add StorageLevel to cacheTable API ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.cacheTable"), From 098a93d9c8e2cc31ef3b43c11d653c2e58da298a Mon Sep 17 00:00:00 2001 From: pgandhi Date: Wed, 27 Sep 2017 13:48:47 -0500 Subject: [PATCH 05/10] [SPARK-21809] : Change Stage Page to use datatables to support sorting columns and searching [SPARK-21809] : Fixed a couple of ui issues; the changes are visible both in webui and SHS when running on yarn. Fixed unit tests. Removed two obsolete unit tests. --- .../spark/ui/static/stagespage-template.html | 2 + .../org/apache/spark/ui/static/taskspages.js | 53 ++- .../one_stage_attempt_json_expectation.json | 66 +++- .../one_stage_json_expectation.json | 66 +++- .../stage_task_list_expectation.json | 142 +++++-- ...multi_attempt_app_json_1__expectation.json | 58 ++- ...multi_attempt_app_json_2__expectation.json | 58 ++- ...k_list_w__offset___length_expectation.json | 352 +++++++++++++++--- ...stage_task_list_w__sortBy_expectation.json | 142 +++++-- ...tBy_short_names___runtime_expectation.json | 142 +++++-- ...rtBy_short_names__runtime_expectation.json | 142 +++++-- ...age_with_accumulable_json_expectation.json | 68 +++- .../org/apache/spark/ui/StagePageSuite.scala | 91 ----- .../org/apache/spark/ui/UISeleniumSuite.scala | 2 +- 14 files changed, 1098 insertions(+), 286 deletions(-) delete mode 100644 core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala 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 34db6b644b31e..aeb27f3600b0a 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 @@ -98,6 +98,8 @@

GC Time Scheduler Delay Task Deserialization Time + Shuffle Read Blocked Time + Shuffle Remote Reads Result Serialization Time Getting Result Time Peak Execution Memory diff --git a/core/src/main/resources/org/apache/spark/ui/static/taskspages.js b/core/src/main/resources/org/apache/spark/ui/static/taskspages.js index 507686a405fc5..33f8c10d99d9d 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/taskspages.js +++ b/core/src/main/resources/org/apache/spark/ui/static/taskspages.js @@ -66,12 +66,12 @@ function StageEndPoint(appId) { ind = words.indexOf("history"); if (ind > 0) { var appId = words[ind + 1]; - var attemptId = words[ind + 4].split("&attempt=").pop(); - var stageIdLen = words[ind + 4].indexOf('&'); - var stageId = words[ind + 4].substr(4, stageIdLen - 4); + var attemptId = words[ind + 5].split("&attempt=").pop(); + var stageIdLen = words[ind + 5].indexOf('&'); + var stageId = words[ind + 5].substr(4, stageIdLen - 4); var newBaseURI = words.slice(0, ind).join('/'); if (isNaN(attemptId) || attemptId == "0") { - return newBaseURI + "/api/v1/applications/" + appId + "/stages/" + stageId; + return newBaseURI + "/api/v1/applications/" + appId + "/" + "1" + "/stages/" + stageId; } else { return newBaseURI + "/api/v1/applications/" + appId + "/" + attemptId + "/stages/" + stageId; } @@ -110,12 +110,14 @@ $(document).ready(function () { " Show Additional Metrics" + "" + ""); tasksSummary = $("#active-tasks"); @@ -372,6 +374,16 @@ $(document).ready(function () { return type === 'display' ? formatDuration(row.taskMetrics.executorDeserializeTime) : row.taskMetrics.executorDeserializeTime; } }, + { + data : function (row, type) { + return type === 'display' ? formatDuration(row.taskMetrics.shuffleReadMetrics.fetchWaitTime) : row.taskMetrics.shuffleReadMetrics.fetchWaitTime; + } + }, + { + data : function (row, type) { + return type === 'display' ? formatBytes(row.taskMetrics.shuffleReadMetrics.remoteBytesRead, type) : row.taskMetrics.shuffleReadMetrics.remoteBytesRead; + } + }, { data : function (row, type) { return type === 'display' ? formatDuration(row.taskMetrics.resultSerializationTime) : row.taskMetrics.resultSerializationTime; @@ -384,7 +396,7 @@ $(document).ready(function () { }, { data : function (row, type) { - return type === 'display' ? formatDuration(row.taskMetrics.peakExecutionMemory) : row.taskMetrics.peakExecutionMemory; + return type === 'display' ? formatBytes(row.taskMetrics.peakExecutionMemory, type) : row.taskMetrics.peakExecutionMemory; } }, { @@ -406,18 +418,33 @@ $(document).ready(function () { { "visible": false, "targets": 11 }, { "visible": false, "targets": 12 }, { "visible": false, "targets": 13 }, - { "visible": false, "targets": 14 } + { "visible": false, "targets": 14 }, + { "visible": false, "targets": 15 }, + { "visible": false, "targets": 16 } ], "order": [[0, "asc"]] }; var taskTableSelector = $(taskTable).DataTable(task_conf); + var optionalColumns = [10, 11, 12, 13, 14, 15, 16]; + var allChecked = true; + for(k = 0; k < optionalColumns.length; k++) { + if (taskTableSelector.column(optionalColumns[k]).visible()) { + document.getElementById("box-"+optionalColumns[k]).checked = true; + } else { + allChecked = false; + } + } + if (allChecked) { + document.getElementById("box-0").checked = true; + } + // hide or show columns dynamically event $('input.toggle-vis').on('click', function(e){ // Get the column var para = $(this).attr('data-column'); if(para == "0"){ - var column = taskTableSelector.column([10, 11, 12, 13, 14]); + var column = taskTableSelector.column([10, 11, 12, 13, 14, 15, 16]); if($(this).is(":checked")){ $(".toggle-vis").prop('checked', true); column.visible(true); 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 6fb40f6f1713b..c34f11986783e 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 @@ -31,10 +31,13 @@ "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.829GMT", "duration" : 435, + "gettingResultTime" : 0.0, + "schedulerDelay" : 16.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -42,6 +45,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 435, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 1902, "jvmGcTime" : 19, "resultSerializationTime" : 2, @@ -69,7 +73,8 @@ "writeTime" : 94000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } }, "9" : { "taskId" : 9, @@ -77,10 +82,13 @@ "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.830GMT", "duration" : 436, + "gettingResultTime" : 0.0, + "schedulerDelay" : 17.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -88,6 +96,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 436, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 1902, "jvmGcTime" : 19, "resultSerializationTime" : 0, @@ -115,7 +124,8 @@ "writeTime" : 98000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } }, "10" : { "taskId" : 10, @@ -123,10 +133,13 @@ "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.830GMT", "duration" : 434, + "gettingResultTime" : 0.0, + "schedulerDelay" : 19.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -134,6 +147,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 434, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 1902, "jvmGcTime" : 19, "resultSerializationTime" : 1, @@ -161,7 +175,8 @@ "writeTime" : 76000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } }, "11" : { "taskId" : 11, @@ -169,10 +184,13 @@ "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.830GMT", "duration" : 434, + "gettingResultTime" : 0.0, + "schedulerDelay" : 17.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -180,6 +198,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 434, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 1902, "jvmGcTime" : 19, "resultSerializationTime" : 1, @@ -207,7 +226,8 @@ "writeTime" : 83000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } }, "12" : { "taskId" : 12, @@ -215,10 +235,13 @@ "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.831GMT", "duration" : 434, + "gettingResultTime" : 0.0, + "schedulerDelay" : 17.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -226,6 +249,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 434, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 1902, "jvmGcTime" : 19, "resultSerializationTime" : 1, @@ -253,7 +277,8 @@ "writeTime" : 101000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } }, "13" : { "taskId" : 13, @@ -261,10 +286,13 @@ "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.831GMT", "duration" : 434, + "gettingResultTime" : 0.0, + "schedulerDelay" : 14.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -272,6 +300,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 434, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 1902, "jvmGcTime" : 19, "resultSerializationTime" : 2, @@ -299,7 +328,8 @@ "writeTime" : 73000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } }, "14" : { "taskId" : 14, @@ -307,10 +337,13 @@ "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.832GMT", "duration" : 434, + "gettingResultTime" : 0.0, + "schedulerDelay" : 13.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -318,6 +351,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 434, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 1902, "jvmGcTime" : 19, "resultSerializationTime" : 1, @@ -345,7 +379,8 @@ "writeTime" : 88000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } }, "15" : { "taskId" : 15, @@ -353,10 +388,13 @@ "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.833GMT", "duration" : 435, + "gettingResultTime" : 0.0, + "schedulerDelay" : 13.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -364,6 +402,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 435, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 1902, "jvmGcTime" : 19, "resultSerializationTime" : 1, @@ -391,7 +430,8 @@ "writeTime" : 79000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } } }, "executorSummary" : { @@ -403,8 +443,16 @@ "outputBytes" : 0, "shuffleRead" : 0, "shuffleWrite" : 13180, + "shuffleWriteRecords" : 0, "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 + "diskBytesSpilled" : 0, + "killedTasks" : 0, + "blacklisted" : 0, + "inputRecords" : 0, + "outputRecords" : 0, + "shuffleReadRecords" : 0, + "host" : "localhost:57971", + "executorLogs" : { } } } } 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 f5a89a2107646..0ba8c185b263a 100644 --- a/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json @@ -31,10 +31,13 @@ "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.829GMT", "duration" : 435, + "gettingResultTime" : 0.0, + "schedulerDelay" : 16.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -42,6 +45,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 435, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 1902, "jvmGcTime" : 19, "resultSerializationTime" : 2, @@ -69,7 +73,8 @@ "writeTime" : 94000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } }, "9" : { "taskId" : 9, @@ -77,10 +82,13 @@ "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.830GMT", "duration" : 436, + "gettingResultTime" : 0.0, + "schedulerDelay" : 17.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -88,6 +96,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 436, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 1902, "jvmGcTime" : 19, "resultSerializationTime" : 0, @@ -115,7 +124,8 @@ "writeTime" : 98000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } }, "10" : { "taskId" : 10, @@ -123,10 +133,13 @@ "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.830GMT", "duration" : 434, + "gettingResultTime" : 0.0, + "schedulerDelay" : 19.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -134,6 +147,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 434, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 1902, "jvmGcTime" : 19, "resultSerializationTime" : 1, @@ -161,7 +175,8 @@ "writeTime" : 76000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } }, "11" : { "taskId" : 11, @@ -169,10 +184,13 @@ "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.830GMT", "duration" : 434, + "gettingResultTime" : 0.0, + "schedulerDelay" : 17.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -180,6 +198,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 434, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 1902, "jvmGcTime" : 19, "resultSerializationTime" : 1, @@ -207,7 +226,8 @@ "writeTime" : 83000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } }, "12" : { "taskId" : 12, @@ -215,10 +235,13 @@ "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.831GMT", "duration" : 434, + "gettingResultTime" : 0.0, + "schedulerDelay" : 17.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -226,6 +249,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 434, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 1902, "jvmGcTime" : 19, "resultSerializationTime" : 1, @@ -253,7 +277,8 @@ "writeTime" : 101000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } }, "13" : { "taskId" : 13, @@ -261,10 +286,13 @@ "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.831GMT", "duration" : 434, + "gettingResultTime" : 0.0, + "schedulerDelay" : 14.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -272,6 +300,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 434, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 1902, "jvmGcTime" : 19, "resultSerializationTime" : 2, @@ -299,7 +328,8 @@ "writeTime" : 73000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } }, "14" : { "taskId" : 14, @@ -307,10 +337,13 @@ "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.832GMT", "duration" : 434, + "gettingResultTime" : 0.0, + "schedulerDelay" : 13.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -318,6 +351,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 434, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 1902, "jvmGcTime" : 19, "resultSerializationTime" : 1, @@ -345,7 +379,8 @@ "writeTime" : 88000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } }, "15" : { "taskId" : 15, @@ -353,10 +388,13 @@ "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.833GMT", "duration" : 435, + "gettingResultTime" : 0.0, + "schedulerDelay" : 13.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -364,6 +402,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 435, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 1902, "jvmGcTime" : 19, "resultSerializationTime" : 1, @@ -391,7 +430,8 @@ "writeTime" : 79000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } } }, "executorSummary" : { @@ -403,8 +443,16 @@ "outputBytes" : 0, "shuffleRead" : 0, "shuffleWrite" : 13180, + "shuffleWriteRecords" : 0, "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 + "diskBytesSpilled" : 0, + "killedTasks" : 0, + "blacklisted" : 0, + "inputRecords" : 0, + "outputRecords" : 0, + "shuffleReadRecords" : 0, + "host" : "localhost:57971", + "executorLogs" : { } } } } ] 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 9b401b414f8d4..25e9296e0da4d 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_expectation.json @@ -4,10 +4,13 @@ "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.494GMT", "duration" : 349, + "gettingResultTime" : 0.0, + "schedulerDelay" : 53.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -15,6 +18,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 349, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 1, @@ -42,17 +46,21 @@ "writeTime" : 3842811, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 1, "index" : 1, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.502GMT", "duration" : 350, + "gettingResultTime" : 0.0, + "schedulerDelay" : 40.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -60,6 +68,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 350, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 0, @@ -87,17 +96,21 @@ "writeTime" : 3934399, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 2, "index" : 2, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.503GMT", "duration" : 348, + "gettingResultTime" : 0.0, + "schedulerDelay" : 37.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -105,6 +118,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 348, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 2, @@ -132,17 +146,21 @@ "writeTime" : 89885, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 3, "index" : 3, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.504GMT", "duration" : 349, + "gettingResultTime" : 0.0, + "schedulerDelay" : 41.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -150,6 +168,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 349, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 2, @@ -177,17 +196,21 @@ "writeTime" : 1311694, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 4, "index" : 4, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.504GMT", "duration" : 349, + "gettingResultTime" : 0.0, + "schedulerDelay" : 38.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -195,6 +218,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 349, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 1, @@ -222,17 +246,21 @@ "writeTime" : 83022, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 5, "index" : 5, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.505GMT", "duration" : 350, + "gettingResultTime" : 0.0, + "schedulerDelay" : 33.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -240,6 +268,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 350, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 1, @@ -267,17 +296,21 @@ "writeTime" : 3675510, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 6, "index" : 6, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.505GMT", "duration" : 351, + "gettingResultTime" : 0.0, + "schedulerDelay" : 38.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -285,6 +318,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 351, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 1, @@ -312,17 +346,21 @@ "writeTime" : 4016617, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 7, "index" : 7, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.506GMT", "duration" : 349, + "gettingResultTime" : 0.0, + "schedulerDelay" : 43.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -330,6 +368,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 349, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 0, @@ -357,17 +396,21 @@ "writeTime" : 2579051, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 8, "index" : 8, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.914GMT", "duration" : 80, + "gettingResultTime" : 0.0, + "schedulerDelay" : 5.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -375,6 +418,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 80, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -402,17 +446,21 @@ "writeTime" : 121551, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 9, "index" : 9, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.915GMT", "duration" : 84, + "gettingResultTime" : 0.0, + "schedulerDelay" : 8.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -420,6 +468,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 84, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -447,17 +496,21 @@ "writeTime" : 101664, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 10, "index" : 10, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.916GMT", "duration" : 73, + "gettingResultTime" : 0.0, + "schedulerDelay" : 18.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -465,6 +518,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 73, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -492,17 +546,21 @@ "writeTime" : 94709, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 11, "index" : 11, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.918GMT", "duration" : 75, + "gettingResultTime" : 0.0, + "schedulerDelay" : 8.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -510,6 +568,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 75, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -537,17 +596,21 @@ "writeTime" : 94507, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 12, "index" : 12, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.923GMT", "duration" : 77, + "gettingResultTime" : 0.0, + "schedulerDelay" : 7.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -555,6 +618,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 77, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -582,17 +646,21 @@ "writeTime" : 102476, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 13, "index" : 13, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.924GMT", "duration" : 76, + "gettingResultTime" : 0.0, + "schedulerDelay" : 53.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -600,6 +668,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 76, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -627,17 +696,21 @@ "writeTime" : 95004, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 14, "index" : 14, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.925GMT", "duration" : 83, + "gettingResultTime" : 0.0, + "schedulerDelay" : 5.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -645,6 +718,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 83, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -672,17 +746,21 @@ "writeTime" : 95646, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 15, "index" : 15, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.928GMT", "duration" : 76, + "gettingResultTime" : 0.0, + "schedulerDelay" : 4.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -690,6 +768,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 76, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -717,17 +796,21 @@ "writeTime" : 602780, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 16, "index" : 16, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.001GMT", "duration" : 84, + "gettingResultTime" : 0.0, + "schedulerDelay" : 4.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -735,6 +818,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 84, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -762,17 +846,21 @@ "writeTime" : 108320, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 17, "index" : 17, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.005GMT", "duration" : 91, + "gettingResultTime" : 0.0, + "schedulerDelay" : 20.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -780,6 +868,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 91, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 5, "resultSerializationTime" : 1, @@ -807,17 +896,21 @@ "writeTime" : 99944, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 18, "index" : 18, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.010GMT", "duration" : 92, + "gettingResultTime" : 0.0, + "schedulerDelay" : 3.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -825,6 +918,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 92, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -852,17 +946,21 @@ "writeTime" : 100836, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 19, "index" : 19, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.012GMT", "duration" : 84, + "gettingResultTime" : 0.0, + "schedulerDelay" : 5.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -870,6 +968,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 84, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -897,5 +996,6 @@ "writeTime" : 95788, "recordsWritten" : 10 } - } -} ] + }, + "executorLogs" : { } +} ] \ No newline at end of file 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 2ebee66a6d7c2..b53458e313c50 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 @@ -4,10 +4,13 @@ "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.515GMT", "duration" : 15, + "gettingResultTime" : 0.0, + "schedulerDelay" : 30.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, @@ -20,6 +23,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, @@ -47,17 +51,21 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } }, { "taskId" : 1, "index" : 1, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.521GMT", "duration" : 15, + "gettingResultTime" : 0.0, + "schedulerDelay" : 22.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, @@ -70,6 +78,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, @@ -97,17 +106,21 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } }, { "taskId" : 2, "index" : 2, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.522GMT", "duration" : 15, + "gettingResultTime" : 0.0, + "schedulerDelay" : 18.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, @@ -120,6 +133,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, @@ -147,17 +161,21 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } }, { "taskId" : 3, "index" : 3, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.522GMT", "duration" : 15, + "gettingResultTime" : 0.0, + "schedulerDelay" : 20.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, @@ -170,6 +188,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, @@ -197,17 +216,21 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } }, { "taskId" : 4, "index" : 4, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.522GMT", "duration" : 15, + "gettingResultTime" : 0.0, + "schedulerDelay" : 24.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, @@ -220,6 +243,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 1, @@ -247,17 +271,21 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } }, { "taskId" : 5, "index" : 5, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.523GMT", "duration" : 15, + "gettingResultTime" : 0.0, + "schedulerDelay" : 23.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, @@ -270,6 +298,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, @@ -297,17 +326,21 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } }, { "taskId" : 6, "index" : 6, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.523GMT", "duration" : 15, + "gettingResultTime" : 0.0, + "schedulerDelay" : 22.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, @@ -320,6 +353,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, @@ -347,17 +381,21 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } }, { "taskId" : 7, "index" : 7, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.524GMT", "duration" : 15, + "gettingResultTime" : 0.0, + "schedulerDelay" : 22.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, @@ -370,6 +408,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, @@ -397,5 +436,6 @@ "writeTime" : 0, "recordsWritten" : 0 } - } -} ] + }, + "executorLogs" : { } +} ] \ No newline at end of file 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 965a31a4104c3..d428289d09221 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 @@ -4,10 +4,13 @@ "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.515GMT", "duration" : 15, + "gettingResultTime" : 0.0, + "schedulerDelay" : 30.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, @@ -20,6 +23,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, @@ -47,17 +51,21 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } }, { "taskId" : 1, "index" : 1, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.521GMT", "duration" : 15, + "gettingResultTime" : 0.0, + "schedulerDelay" : 22.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, @@ -70,6 +78,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, @@ -97,17 +106,21 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } }, { "taskId" : 2, "index" : 2, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.522GMT", "duration" : 15, + "gettingResultTime" : 0.0, + "schedulerDelay" : 18.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, @@ -120,6 +133,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, @@ -147,17 +161,21 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } }, { "taskId" : 3, "index" : 3, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.522GMT", "duration" : 15, + "gettingResultTime" : 0.0, + "schedulerDelay" : 20.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, @@ -170,6 +188,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, @@ -197,17 +216,21 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } }, { "taskId" : 4, "index" : 4, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.522GMT", "duration" : 15, + "gettingResultTime" : 0.0, + "schedulerDelay" : 24.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, @@ -220,6 +243,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 1, @@ -247,17 +271,21 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } }, { "taskId" : 5, "index" : 5, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.523GMT", "duration" : 15, + "gettingResultTime" : 0.0, + "schedulerDelay" : 23.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, @@ -270,6 +298,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, @@ -297,17 +326,21 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } }, { "taskId" : 6, "index" : 6, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.523GMT", "duration" : 15, + "gettingResultTime" : 0.0, + "schedulerDelay" : 22.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, @@ -320,6 +353,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, @@ -347,17 +381,21 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } }, { "taskId" : 7, "index" : 7, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.524GMT", "duration" : 15, + "gettingResultTime" : 0.0, + "schedulerDelay" : 22.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, @@ -370,6 +408,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, @@ -397,5 +436,6 @@ "writeTime" : 0, "recordsWritten" : 0 } - } -} ] + }, + "executorLogs" : { } +} ] \ No newline at end of file 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 31132e156937c..ff3d2a3285cf4 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 @@ -4,10 +4,13 @@ "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.916GMT", "duration" : 73, + "gettingResultTime" : 0.0, + "schedulerDelay" : 18.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -15,6 +18,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 73, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -42,17 +46,21 @@ "writeTime" : 94709, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 11, "index" : 11, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.918GMT", "duration" : 75, + "gettingResultTime" : 0.0, + "schedulerDelay" : 8.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -60,6 +68,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 75, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -87,17 +96,21 @@ "writeTime" : 94507, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 12, "index" : 12, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.923GMT", "duration" : 77, + "gettingResultTime" : 0.0, + "schedulerDelay" : 7.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -105,6 +118,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 77, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -132,17 +146,21 @@ "writeTime" : 102476, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 13, "index" : 13, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.924GMT", "duration" : 76, + "gettingResultTime" : 0.0, + "schedulerDelay" : 53.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -150,6 +168,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 76, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -177,17 +196,21 @@ "writeTime" : 95004, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 14, "index" : 14, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.925GMT", "duration" : 83, + "gettingResultTime" : 0.0, + "schedulerDelay" : 5.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -195,6 +218,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 83, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -222,17 +246,21 @@ "writeTime" : 95646, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 15, "index" : 15, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.928GMT", "duration" : 76, + "gettingResultTime" : 0.0, + "schedulerDelay" : 4.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -240,6 +268,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 76, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -267,17 +296,21 @@ "writeTime" : 602780, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 16, "index" : 16, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.001GMT", "duration" : 84, + "gettingResultTime" : 0.0, + "schedulerDelay" : 4.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -285,6 +318,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 84, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -312,17 +346,21 @@ "writeTime" : 108320, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 17, "index" : 17, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.005GMT", "duration" : 91, + "gettingResultTime" : 0.0, + "schedulerDelay" : 20.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -330,6 +368,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 91, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 5, "resultSerializationTime" : 1, @@ -357,17 +396,21 @@ "writeTime" : 99944, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 18, "index" : 18, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.010GMT", "duration" : 92, + "gettingResultTime" : 0.0, + "schedulerDelay" : 3.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -375,6 +418,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 92, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -402,17 +446,21 @@ "writeTime" : 100836, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 19, "index" : 19, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.012GMT", "duration" : 84, + "gettingResultTime" : 0.0, + "schedulerDelay" : 5.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -420,6 +468,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 84, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -447,17 +496,21 @@ "writeTime" : 95788, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 20, "index" : 20, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.014GMT", "duration" : 83, + "gettingResultTime" : 0.0, + "schedulerDelay" : 4.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -465,6 +518,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 83, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -492,17 +546,21 @@ "writeTime" : 97716, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 21, "index" : 21, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.015GMT", "duration" : 88, + "gettingResultTime" : 0.0, + "schedulerDelay" : 3.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -510,6 +568,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 88, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -537,17 +596,21 @@ "writeTime" : 100270, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 22, "index" : 22, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.018GMT", "duration" : 93, + "gettingResultTime" : 0.0, + "schedulerDelay" : 3.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -555,6 +618,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 93, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -582,17 +646,21 @@ "writeTime" : 143427, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 23, "index" : 23, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.031GMT", "duration" : 65, + "gettingResultTime" : 0.0, + "schedulerDelay" : 16.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -600,6 +668,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 65, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -627,17 +696,21 @@ "writeTime" : 91844, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 24, "index" : 24, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.098GMT", "duration" : 43, + "gettingResultTime" : 0.0, + "schedulerDelay" : 5.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -645,6 +718,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 43, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 1, @@ -672,17 +746,21 @@ "writeTime" : 157194, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 25, "index" : 25, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.103GMT", "duration" : 49, + "gettingResultTime" : 0.0, + "schedulerDelay" : 9.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -690,6 +768,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 49, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -717,17 +796,21 @@ "writeTime" : 94134, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 26, "index" : 26, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.105GMT", "duration" : 38, + "gettingResultTime" : 0.0, + "schedulerDelay" : 8.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -735,6 +818,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 38, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -762,17 +846,21 @@ "writeTime" : 108213, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 27, "index" : 27, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.110GMT", "duration" : 32, + "gettingResultTime" : 0.0, + "schedulerDelay" : 6.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -780,6 +868,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 32, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -807,17 +896,21 @@ "writeTime" : 102019, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 28, "index" : 28, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.113GMT", "duration" : 29, + "gettingResultTime" : 0.0, + "schedulerDelay" : 17.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -825,6 +918,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 29, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -852,17 +946,21 @@ "writeTime" : 104299, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 29, "index" : 29, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.114GMT", "duration" : 39, + "gettingResultTime" : 0.0, + "schedulerDelay" : 11.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -870,6 +968,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 39, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -897,17 +996,21 @@ "writeTime" : 114938, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 30, "index" : 30, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.118GMT", "duration" : 34, + "gettingResultTime" : 0.0, + "schedulerDelay" : 24.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -915,6 +1018,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 34, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -942,17 +1046,21 @@ "writeTime" : 119770, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 31, "index" : 31, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.127GMT", "duration" : 24, + "gettingResultTime" : 0.0, + "schedulerDelay" : 14.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -960,6 +1068,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 24, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -987,17 +1096,21 @@ "writeTime" : 92619, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 32, "index" : 32, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.148GMT", "duration" : 17, + "gettingResultTime" : 0.0, + "schedulerDelay" : 13.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -1005,6 +1118,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 17, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -1032,17 +1146,21 @@ "writeTime" : 89603, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 33, "index" : 33, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.149GMT", "duration" : 43, + "gettingResultTime" : 0.0, + "schedulerDelay" : 10.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -1050,6 +1168,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 43, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -1077,17 +1196,21 @@ "writeTime" : 118329, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 34, "index" : 34, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.156GMT", "duration" : 27, + "gettingResultTime" : 0.0, + "schedulerDelay" : 9.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -1095,6 +1218,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 27, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -1122,17 +1246,21 @@ "writeTime" : 127746, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 35, "index" : 35, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.161GMT", "duration" : 35, + "gettingResultTime" : 0.0, + "schedulerDelay" : 10.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -1140,6 +1268,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 35, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -1167,17 +1296,21 @@ "writeTime" : 160963, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 36, "index" : 36, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.164GMT", "duration" : 29, + "gettingResultTime" : 0.0, + "schedulerDelay" : 6.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -1185,6 +1318,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 29, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -1212,17 +1346,21 @@ "writeTime" : 123855, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 37, "index" : 37, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.165GMT", "duration" : 32, + "gettingResultTime" : 0.0, + "schedulerDelay" : 6.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -1230,6 +1368,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 32, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -1257,17 +1396,21 @@ "writeTime" : 111869, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 38, "index" : 38, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.166GMT", "duration" : 31, + "gettingResultTime" : 0.0, + "schedulerDelay" : 13.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -1275,6 +1418,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 31, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -1302,17 +1446,21 @@ "writeTime" : 131158, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 39, "index" : 39, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.180GMT", "duration" : 17, + "gettingResultTime" : 0.0, + "schedulerDelay" : 13.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -1320,6 +1468,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 17, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -1347,17 +1496,21 @@ "writeTime" : 98748, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 40, "index" : 40, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.197GMT", "duration" : 14, + "gettingResultTime" : 0.0, + "schedulerDelay" : 6.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -1365,6 +1518,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 14, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -1392,17 +1546,21 @@ "writeTime" : 94792, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 41, "index" : 41, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.200GMT", "duration" : 16, + "gettingResultTime" : 0.0, + "schedulerDelay" : 6.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -1410,6 +1568,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 16, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -1437,17 +1596,21 @@ "writeTime" : 90765, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 42, "index" : 42, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.203GMT", "duration" : 17, + "gettingResultTime" : 0.0, + "schedulerDelay" : 15.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -1455,6 +1618,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 17, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -1482,17 +1646,21 @@ "writeTime" : 103713, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 43, "index" : 43, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.204GMT", "duration" : 16, + "gettingResultTime" : 0.0, + "schedulerDelay" : 21.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -1500,6 +1668,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 16, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -1527,17 +1696,21 @@ "writeTime" : 171516, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 44, "index" : 44, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.205GMT", "duration" : 18, + "gettingResultTime" : 0.0, + "schedulerDelay" : 16.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -1545,6 +1718,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 18, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -1572,17 +1746,21 @@ "writeTime" : 98293, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 45, "index" : 45, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.206GMT", "duration" : 19, + "gettingResultTime" : 0.0, + "schedulerDelay" : 16.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -1590,6 +1768,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 19, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -1617,17 +1796,21 @@ "writeTime" : 92985, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 46, "index" : 46, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.210GMT", "duration" : 31, + "gettingResultTime" : 0.0, + "schedulerDelay" : 11.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -1635,6 +1818,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 31, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 6, "resultSerializationTime" : 0, @@ -1662,17 +1846,21 @@ "writeTime" : 113322, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 47, "index" : 47, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.212GMT", "duration" : 18, + "gettingResultTime" : 0.0, + "schedulerDelay" : 13.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -1680,6 +1868,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 18, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -1707,17 +1896,21 @@ "writeTime" : 103015, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 48, "index" : 48, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.220GMT", "duration" : 24, + "gettingResultTime" : 0.0, + "schedulerDelay" : 3.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -1725,6 +1918,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 24, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 6, "resultSerializationTime" : 0, @@ -1752,17 +1946,21 @@ "writeTime" : 139844, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 49, "index" : 49, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.223GMT", "duration" : 23, + "gettingResultTime" : 0.0, + "schedulerDelay" : 4.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -1770,6 +1968,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 23, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 6, "resultSerializationTime" : 0, @@ -1797,17 +1996,21 @@ "writeTime" : 94984, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 50, "index" : 50, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.240GMT", "duration" : 18, + "gettingResultTime" : 0.0, + "schedulerDelay" : 4.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -1815,6 +2018,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 18, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -1842,17 +2046,21 @@ "writeTime" : 90836, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 51, "index" : 51, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.242GMT", "duration" : 17, + "gettingResultTime" : 0.0, + "schedulerDelay" : 2.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -1860,6 +2068,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 17, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -1887,17 +2096,21 @@ "writeTime" : 96013, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 52, "index" : 52, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.243GMT", "duration" : 18, + "gettingResultTime" : 0.0, + "schedulerDelay" : 5.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -1905,6 +2118,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 18, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -1932,17 +2146,21 @@ "writeTime" : 89664, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 53, "index" : 53, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.244GMT", "duration" : 18, + "gettingResultTime" : 0.0, + "schedulerDelay" : 5.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -1950,6 +2168,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 18, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -1977,17 +2196,21 @@ "writeTime" : 92835, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 54, "index" : 54, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.244GMT", "duration" : 18, + "gettingResultTime" : 0.0, + "schedulerDelay" : 38.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -1995,6 +2218,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 18, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -2022,17 +2246,21 @@ "writeTime" : 90506, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 55, "index" : 55, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.246GMT", "duration" : 21, + "gettingResultTime" : 0.0, + "schedulerDelay" : 5.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -2040,6 +2268,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 21, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -2067,17 +2296,21 @@ "writeTime" : 108309, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 56, "index" : 56, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.249GMT", "duration" : 20, + "gettingResultTime" : 0.0, + "schedulerDelay" : 6.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -2085,6 +2318,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 20, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -2112,17 +2346,21 @@ "writeTime" : 90329, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 57, "index" : 57, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.257GMT", "duration" : 16, + "gettingResultTime" : 0.0, + "schedulerDelay" : 2.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -2130,6 +2368,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 16, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -2157,17 +2396,21 @@ "writeTime" : 96849, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 58, "index" : 58, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.263GMT", "duration" : 16, + "gettingResultTime" : 0.0, + "schedulerDelay" : 4.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -2175,6 +2418,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 16, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -2202,17 +2446,21 @@ "writeTime" : 97521, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 59, "index" : 59, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.265GMT", "duration" : 17, + "gettingResultTime" : 0.0, + "schedulerDelay" : 3.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -2220,6 +2468,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 17, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -2247,5 +2496,6 @@ "writeTime" : 100753, "recordsWritten" : 10 } - } -} ] + }, + "executorLogs" : { } +} ] \ No newline at end of file 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 6af1cfbeb8f7e..131a747d6cf02 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 @@ -4,10 +4,13 @@ "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.505GMT", "duration" : 351, + "gettingResultTime" : 0.0, + "schedulerDelay" : 38.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -15,6 +18,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 351, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 1, @@ -42,17 +46,21 @@ "writeTime" : 4016617, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 1, "index" : 1, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.502GMT", "duration" : 350, + "gettingResultTime" : 0.0, + "schedulerDelay" : 40.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -60,6 +68,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 350, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 0, @@ -87,17 +96,21 @@ "writeTime" : 3934399, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 5, "index" : 5, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.505GMT", "duration" : 350, + "gettingResultTime" : 0.0, + "schedulerDelay" : 33.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -105,6 +118,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 350, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 1, @@ -132,17 +146,21 @@ "writeTime" : 3675510, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 0, "index" : 0, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.494GMT", "duration" : 349, + "gettingResultTime" : 0.0, + "schedulerDelay" : 53.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -150,6 +168,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 349, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 1, @@ -177,17 +196,21 @@ "writeTime" : 3842811, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 3, "index" : 3, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.504GMT", "duration" : 349, + "gettingResultTime" : 0.0, + "schedulerDelay" : 41.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -195,6 +218,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 349, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 2, @@ -222,17 +246,21 @@ "writeTime" : 1311694, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 4, "index" : 4, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.504GMT", "duration" : 349, + "gettingResultTime" : 0.0, + "schedulerDelay" : 38.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -240,6 +268,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 349, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 1, @@ -267,17 +296,21 @@ "writeTime" : 83022, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 7, "index" : 7, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.506GMT", "duration" : 349, + "gettingResultTime" : 0.0, + "schedulerDelay" : 43.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -285,6 +318,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 349, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 0, @@ -312,17 +346,21 @@ "writeTime" : 2579051, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 2, "index" : 2, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.503GMT", "duration" : 348, + "gettingResultTime" : 0.0, + "schedulerDelay" : 37.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -330,6 +368,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 348, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 2, @@ -357,17 +396,21 @@ "writeTime" : 89885, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 22, "index" : 22, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.018GMT", "duration" : 93, + "gettingResultTime" : 0.0, + "schedulerDelay" : 3.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -375,6 +418,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 93, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -402,17 +446,21 @@ "writeTime" : 143427, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 18, "index" : 18, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.010GMT", "duration" : 92, + "gettingResultTime" : 0.0, + "schedulerDelay" : 3.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -420,6 +468,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 92, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -447,17 +496,21 @@ "writeTime" : 100836, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 17, "index" : 17, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.005GMT", "duration" : 91, + "gettingResultTime" : 0.0, + "schedulerDelay" : 20.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -465,6 +518,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 91, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 5, "resultSerializationTime" : 1, @@ -492,17 +546,21 @@ "writeTime" : 99944, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 21, "index" : 21, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.015GMT", "duration" : 88, + "gettingResultTime" : 0.0, + "schedulerDelay" : 3.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -510,6 +568,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 88, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -537,17 +596,21 @@ "writeTime" : 100270, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 9, "index" : 9, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.915GMT", "duration" : 84, + "gettingResultTime" : 0.0, + "schedulerDelay" : 8.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -555,6 +618,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 84, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -582,17 +646,21 @@ "writeTime" : 101664, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 16, "index" : 16, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.001GMT", "duration" : 84, + "gettingResultTime" : 0.0, + "schedulerDelay" : 4.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -600,6 +668,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 84, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -627,17 +696,21 @@ "writeTime" : 108320, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 19, "index" : 19, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.012GMT", "duration" : 84, + "gettingResultTime" : 0.0, + "schedulerDelay" : 5.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -645,6 +718,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 84, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -672,17 +746,21 @@ "writeTime" : 95788, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 14, "index" : 14, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.925GMT", "duration" : 83, + "gettingResultTime" : 0.0, + "schedulerDelay" : 5.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -690,6 +768,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 83, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -717,17 +796,21 @@ "writeTime" : 95646, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 20, "index" : 20, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.014GMT", "duration" : 83, + "gettingResultTime" : 0.0, + "schedulerDelay" : 4.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -735,6 +818,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 83, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -762,17 +846,21 @@ "writeTime" : 97716, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 8, "index" : 8, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.914GMT", "duration" : 80, + "gettingResultTime" : 0.0, + "schedulerDelay" : 5.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -780,6 +868,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 80, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -807,17 +896,21 @@ "writeTime" : 121551, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 12, "index" : 12, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.923GMT", "duration" : 77, + "gettingResultTime" : 0.0, + "schedulerDelay" : 7.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -825,6 +918,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 77, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -852,17 +946,21 @@ "writeTime" : 102476, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 13, "index" : 13, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.924GMT", "duration" : 76, + "gettingResultTime" : 0.0, + "schedulerDelay" : 53.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -870,6 +968,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 76, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -897,5 +996,6 @@ "writeTime" : 95004, "recordsWritten" : 10 } - } -} ] + }, + "executorLogs" : { } +} ] \ No newline at end of file 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 6af1cfbeb8f7e..131a747d6cf02 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 @@ -4,10 +4,13 @@ "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.505GMT", "duration" : 351, + "gettingResultTime" : 0.0, + "schedulerDelay" : 38.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -15,6 +18,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 351, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 1, @@ -42,17 +46,21 @@ "writeTime" : 4016617, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 1, "index" : 1, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.502GMT", "duration" : 350, + "gettingResultTime" : 0.0, + "schedulerDelay" : 40.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -60,6 +68,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 350, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 0, @@ -87,17 +96,21 @@ "writeTime" : 3934399, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 5, "index" : 5, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.505GMT", "duration" : 350, + "gettingResultTime" : 0.0, + "schedulerDelay" : 33.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -105,6 +118,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 350, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 1, @@ -132,17 +146,21 @@ "writeTime" : 3675510, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 0, "index" : 0, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.494GMT", "duration" : 349, + "gettingResultTime" : 0.0, + "schedulerDelay" : 53.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -150,6 +168,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 349, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 1, @@ -177,17 +196,21 @@ "writeTime" : 3842811, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 3, "index" : 3, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.504GMT", "duration" : 349, + "gettingResultTime" : 0.0, + "schedulerDelay" : 41.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -195,6 +218,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 349, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 2, @@ -222,17 +246,21 @@ "writeTime" : 1311694, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 4, "index" : 4, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.504GMT", "duration" : 349, + "gettingResultTime" : 0.0, + "schedulerDelay" : 38.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -240,6 +268,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 349, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 1, @@ -267,17 +296,21 @@ "writeTime" : 83022, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 7, "index" : 7, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.506GMT", "duration" : 349, + "gettingResultTime" : 0.0, + "schedulerDelay" : 43.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -285,6 +318,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 349, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 0, @@ -312,17 +346,21 @@ "writeTime" : 2579051, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 2, "index" : 2, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.503GMT", "duration" : 348, + "gettingResultTime" : 0.0, + "schedulerDelay" : 37.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -330,6 +368,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 348, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 2, @@ -357,17 +396,21 @@ "writeTime" : 89885, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 22, "index" : 22, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.018GMT", "duration" : 93, + "gettingResultTime" : 0.0, + "schedulerDelay" : 3.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -375,6 +418,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 93, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -402,17 +446,21 @@ "writeTime" : 143427, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 18, "index" : 18, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.010GMT", "duration" : 92, + "gettingResultTime" : 0.0, + "schedulerDelay" : 3.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -420,6 +468,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 92, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -447,17 +496,21 @@ "writeTime" : 100836, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 17, "index" : 17, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.005GMT", "duration" : 91, + "gettingResultTime" : 0.0, + "schedulerDelay" : 20.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -465,6 +518,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 91, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 5, "resultSerializationTime" : 1, @@ -492,17 +546,21 @@ "writeTime" : 99944, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 21, "index" : 21, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.015GMT", "duration" : 88, + "gettingResultTime" : 0.0, + "schedulerDelay" : 3.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -510,6 +568,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 88, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -537,17 +596,21 @@ "writeTime" : 100270, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 9, "index" : 9, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.915GMT", "duration" : 84, + "gettingResultTime" : 0.0, + "schedulerDelay" : 8.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -555,6 +618,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 84, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -582,17 +646,21 @@ "writeTime" : 101664, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 16, "index" : 16, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.001GMT", "duration" : 84, + "gettingResultTime" : 0.0, + "schedulerDelay" : 4.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -600,6 +668,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 84, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -627,17 +696,21 @@ "writeTime" : 108320, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 19, "index" : 19, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.012GMT", "duration" : 84, + "gettingResultTime" : 0.0, + "schedulerDelay" : 5.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -645,6 +718,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 84, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -672,17 +746,21 @@ "writeTime" : 95788, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 14, "index" : 14, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.925GMT", "duration" : 83, + "gettingResultTime" : 0.0, + "schedulerDelay" : 5.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -690,6 +768,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 83, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -717,17 +796,21 @@ "writeTime" : 95646, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 20, "index" : 20, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.014GMT", "duration" : 83, + "gettingResultTime" : 0.0, + "schedulerDelay" : 4.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -735,6 +818,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 83, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -762,17 +846,21 @@ "writeTime" : 97716, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 8, "index" : 8, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.914GMT", "duration" : 80, + "gettingResultTime" : 0.0, + "schedulerDelay" : 5.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -780,6 +868,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 80, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -807,17 +896,21 @@ "writeTime" : 121551, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 12, "index" : 12, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.923GMT", "duration" : 77, + "gettingResultTime" : 0.0, + "schedulerDelay" : 7.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -825,6 +918,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 77, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -852,17 +946,21 @@ "writeTime" : 102476, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 13, "index" : 13, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.924GMT", "duration" : 76, + "gettingResultTime" : 0.0, + "schedulerDelay" : 53.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -870,6 +968,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 76, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -897,5 +996,6 @@ "writeTime" : 95004, "recordsWritten" : 10 } - } -} ] + }, + "executorLogs" : { } +} ] \ No newline at end of file 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 c26daf4b8d7bd..f94d7106e0725 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 @@ -4,10 +4,13 @@ "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.197GMT", "duration" : 14, + "gettingResultTime" : 0.0, + "schedulerDelay" : 6.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -15,6 +18,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 14, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -42,17 +46,21 @@ "writeTime" : 94792, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 41, "index" : 41, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.200GMT", "duration" : 16, + "gettingResultTime" : 0.0, + "schedulerDelay" : 6.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -60,6 +68,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 16, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -87,17 +96,21 @@ "writeTime" : 90765, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 43, "index" : 43, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.204GMT", "duration" : 16, + "gettingResultTime" : 0.0, + "schedulerDelay" : 21.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -105,6 +118,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 16, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -132,17 +146,21 @@ "writeTime" : 171516, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 57, "index" : 57, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.257GMT", "duration" : 16, + "gettingResultTime" : 0.0, + "schedulerDelay" : 2.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -150,6 +168,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 16, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -177,17 +196,21 @@ "writeTime" : 96849, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 58, "index" : 58, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.263GMT", "duration" : 16, + "gettingResultTime" : 0.0, + "schedulerDelay" : 4.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -195,6 +218,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 16, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -222,17 +246,21 @@ "writeTime" : 97521, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 68, "index" : 68, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.306GMT", "duration" : 16, + "gettingResultTime" : 0.0, + "schedulerDelay" : 4.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -240,6 +268,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 16, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -267,17 +296,21 @@ "writeTime" : 101750, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 86, "index" : 86, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.374GMT", "duration" : 16, + "gettingResultTime" : 0.0, + "schedulerDelay" : 8.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -285,6 +318,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 16, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 1, @@ -312,17 +346,21 @@ "writeTime" : 95848, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 32, "index" : 32, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.148GMT", "duration" : 17, + "gettingResultTime" : 0.0, + "schedulerDelay" : 13.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -330,6 +368,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 17, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -357,17 +396,21 @@ "writeTime" : 89603, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 39, "index" : 39, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.180GMT", "duration" : 17, + "gettingResultTime" : 0.0, + "schedulerDelay" : 13.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -375,6 +418,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 17, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -402,17 +446,21 @@ "writeTime" : 98748, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 42, "index" : 42, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.203GMT", "duration" : 17, + "gettingResultTime" : 0.0, + "schedulerDelay" : 15.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -420,6 +468,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 17, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -447,17 +496,21 @@ "writeTime" : 103713, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 51, "index" : 51, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.242GMT", "duration" : 17, + "gettingResultTime" : 0.0, + "schedulerDelay" : 2.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -465,6 +518,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 17, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -492,17 +546,21 @@ "writeTime" : 96013, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 59, "index" : 59, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.265GMT", "duration" : 17, + "gettingResultTime" : 0.0, + "schedulerDelay" : 3.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -510,6 +568,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 17, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -537,17 +596,21 @@ "writeTime" : 100753, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 63, "index" : 63, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.276GMT", "duration" : 17, + "gettingResultTime" : 0.0, + "schedulerDelay" : 3.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -555,6 +618,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 17, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -582,17 +646,21 @@ "writeTime" : 102779, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 87, "index" : 87, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.374GMT", "duration" : 17, + "gettingResultTime" : 0.0, + "schedulerDelay" : 7.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -600,6 +668,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 17, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -627,17 +696,21 @@ "writeTime" : 102159, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 90, "index" : 90, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.385GMT", "duration" : 17, + "gettingResultTime" : 0.0, + "schedulerDelay" : 4.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -645,6 +718,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 17, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -672,17 +746,21 @@ "writeTime" : 98472, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 99, "index" : 99, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.426GMT", "duration" : 17, + "gettingResultTime" : 0.0, + "schedulerDelay" : 3.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -690,6 +768,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 17, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -717,17 +796,21 @@ "writeTime" : 133964, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 44, "index" : 44, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.205GMT", "duration" : 18, + "gettingResultTime" : 0.0, + "schedulerDelay" : 16.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -735,6 +818,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 18, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -762,17 +846,21 @@ "writeTime" : 98293, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 47, "index" : 47, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.212GMT", "duration" : 18, + "gettingResultTime" : 0.0, + "schedulerDelay" : 13.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -780,6 +868,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 18, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -807,17 +896,21 @@ "writeTime" : 103015, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 50, "index" : 50, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.240GMT", "duration" : 18, + "gettingResultTime" : 0.0, + "schedulerDelay" : 4.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -825,6 +918,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 18, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -852,17 +946,21 @@ "writeTime" : 90836, "recordsWritten" : 10 } - } + }, + "executorLogs" : { } }, { "taskId" : 52, "index" : 52, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.243GMT", "duration" : 18, + "gettingResultTime" : 0.0, + "schedulerDelay" : 5.0, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { @@ -870,6 +968,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 18, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -897,5 +996,6 @@ "writeTime" : 89664, "recordsWritten" : 10 } - } -} ] + }, + "executorLogs" : { } +} ] \ No newline at end of file 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 44b5f66efe339..b105516d1bf79 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 @@ -35,10 +35,13 @@ "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.515GMT", "duration" : 15, + "gettingResultTime" : 0.0, + "schedulerDelay" : 30.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, @@ -51,6 +54,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, @@ -78,7 +82,8 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } }, "1" : { "taskId" : 1, @@ -86,10 +91,13 @@ "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.521GMT", "duration" : 15, + "gettingResultTime" : 0.0, + "schedulerDelay" : 22.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, @@ -102,6 +110,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, @@ -129,7 +138,8 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } }, "2" : { "taskId" : 2, @@ -137,10 +147,13 @@ "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.522GMT", "duration" : 15, + "gettingResultTime" : 0.0, + "schedulerDelay" : 18.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, @@ -153,6 +166,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, @@ -180,7 +194,8 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } }, "3" : { "taskId" : 3, @@ -188,10 +203,13 @@ "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.522GMT", "duration" : 15, + "gettingResultTime" : 0.0, + "schedulerDelay" : 20.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, @@ -204,6 +222,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, @@ -231,7 +250,8 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } }, "4" : { "taskId" : 4, @@ -239,10 +259,13 @@ "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.522GMT", "duration" : 15, + "gettingResultTime" : 0.0, + "schedulerDelay" : 24.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, @@ -255,6 +278,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 1, @@ -282,7 +306,8 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } }, "5" : { "taskId" : 5, @@ -290,10 +315,13 @@ "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.523GMT", "duration" : 15, + "gettingResultTime" : 0.0, + "schedulerDelay" : 23.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, @@ -306,6 +334,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, @@ -333,7 +362,8 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } }, "6" : { "taskId" : 6, @@ -341,10 +371,13 @@ "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.523GMT", "duration" : 15, + "gettingResultTime" : 0.0, + "schedulerDelay" : 22.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, @@ -357,6 +390,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, @@ -384,7 +418,8 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } }, "7" : { "taskId" : 7, @@ -392,10 +427,13 @@ "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.524GMT", "duration" : 15, + "gettingResultTime" : 0.0, + "schedulerDelay" : 22.0, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", + "taskState" : "SUCCESS", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, @@ -408,6 +446,7 @@ "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, + "peakExecutionMemory" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, @@ -435,7 +474,8 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { } } }, "executorSummary" : { @@ -447,8 +487,16 @@ "outputBytes" : 0, "shuffleRead" : 0, "shuffleWrite" : 0, + "shuffleWriteRecords" : 0, "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 + "diskBytesSpilled" : 0, + "killedTasks" : 0, + "blacklisted" : 0, + "inputRecords" : 0, + "outputRecords" : 0, + "shuffleReadRecords" : 0, + "host" : "localhost:58610", + "executorLogs" : { } } } -} +} \ No newline at end of file diff --git a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala deleted file mode 100644 index 499d47b13d702..0000000000000 --- a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala +++ /dev/null @@ -1,91 +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 - -import java.util.Locale -import javax.servlet.http.HttpServletRequest - -import scala.xml.Node - -import org.mockito.Mockito.{mock, when, RETURNS_SMART_NULLS} - -import org.apache.spark._ -import org.apache.spark.executor.TaskMetrics -import org.apache.spark.scheduler._ -import org.apache.spark.storage.StorageStatusListener -import org.apache.spark.ui.exec.ExecutorsListener -import org.apache.spark.ui.jobs.{JobProgressListener, StagePage, StagesTab} -import org.apache.spark.ui.scope.RDDOperationGraphListener - -class StagePageSuite extends SparkFunSuite with LocalSparkContext { - - private val peakExecutionMemory = 10 - - test("peak execution memory should displayed") { - val conf = new SparkConf(false) - val html = renderStagePage(conf).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 conf = new SparkConf(false) - val html = renderStagePage(conf).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. - */ - private def renderStagePage(conf: SparkConf): Seq[Node] = { - val jobListener = new JobProgressListener(conf) - val graphListener = new RDDOperationGraphListener(conf) - val executorsListener = new ExecutorsListener(new StorageStatusListener(conf), conf) - val tab = mock(classOf[StagesTab], RETURNS_SMART_NULLS) - val request = mock(classOf[HttpServletRequest]) - when(tab.conf).thenReturn(conf) - when(tab.progressListener).thenReturn(jobListener) - when(tab.operationGraphListener).thenReturn(graphListener) - when(tab.executorsListener).thenReturn(executorsListener) - when(tab.appName).thenReturn("testing") - when(tab.headerTabs).thenReturn(Seq.empty) - when(request.getParameter("id")).thenReturn("0") - when(request.getParameter("attempt")).thenReturn("0") - val page = new StagePage(tab) - - // Simulate a stage in job progress listener - val stageInfo = new StageInfo(0, 0, "dummy", 1, Seq.empty, Seq.empty, "details") - // Simulate two tasks to test PEAK_EXECUTION_MEMORY correctness - (1 to 2).foreach { - taskId => - val taskInfo = new TaskInfo(taskId, taskId, 0, 0, "0", "localhost", TaskLocality.ANY, false) - jobListener.onStageSubmitted(SparkListenerStageSubmitted(stageInfo)) - jobListener.onTaskStart(SparkListenerTaskStart(0, 0, taskInfo)) - taskInfo.markFinished(TaskState.FINISHED, System.currentTimeMillis()) - val taskMetrics = TaskMetrics.empty - taskMetrics.incPeakExecutionMemory(peakExecutionMemory) - jobListener.onTaskEnd( - SparkListenerTaskEnd(0, 0, "result", Success, taskInfo, taskMetrics)) - } - jobListener.onStageCompleted(SparkListenerStageCompleted(stageInfo)) - page.render(request) - } - -} diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index 267c8dc1bd750..153111af3983b 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -346,7 +346,7 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B for { stageId <- 0 to 1 - attemptId <- 0 to 1 + attemptId <- 1 to 0 } { val exp = if (attemptId == 0 && stageId == 1) StageStatus.FAILED else StageStatus.COMPLETE val stageJson = getJson(sc.ui.get, s"stages/$stageId/$attemptId") From c1f85aeab03e2d172261431d6e5e46cbd303e069 Mon Sep 17 00:00:00 2001 From: pgandhi Date: Fri, 29 Sep 2017 10:35:42 -0500 Subject: [PATCH 06/10] [SPARK-21809] : Fixing issue of not being able to see changes in web ui and history on local and standalone [SPARK-21809] : Fixing issue of not being able to see changes in web ui and history on local and standalone mode as well as on yarn --- .../org/apache/spark/ui/static/taskspages.js | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/taskspages.js b/core/src/main/resources/org/apache/spark/ui/static/taskspages.js index 33f8c10d99d9d..29c2ef70fe1f7 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/taskspages.js +++ b/core/src/main/resources/org/apache/spark/ui/static/taskspages.js @@ -55,29 +55,30 @@ function createTemplateURI(appId) { // e.g. https://axonitered-jt1.red.ygrid.yahoo.com:50509/history/application_1502220952225_59143/stages/stage/?id=0&attempt=0 function StageEndPoint(appId) { var words = document.baseURI.split('/'); + var words2 = document.baseURI.split('?'); var ind = words.indexOf("proxy"); if (ind > 0) { var appId = words[ind + 1]; - var stageIdLen = words[ind + 3].indexOf('&'); - var stageId = words[ind + 3].substr(9, stageIdLen - 9); + var stageIdLen = words2[1].indexOf('&'); + var stageId = words2[1].substr(3, stageIdLen - 3); var newBaseURI = words.slice(0, ind + 2).join('/'); return newBaseURI + "/api/v1/applications/" + appId + "/stages/" + stageId; } ind = words.indexOf("history"); if (ind > 0) { var appId = words[ind + 1]; - var attemptId = words[ind + 5].split("&attempt=").pop(); - var stageIdLen = words[ind + 5].indexOf('&'); - var stageId = words[ind + 5].substr(4, stageIdLen - 4); + var attemptId = words[ind + 2]; + var stageIdLen = words2[1].indexOf('&'); + var stageId = words2[1].substr(3, stageIdLen - 3); var newBaseURI = words.slice(0, ind).join('/'); if (isNaN(attemptId) || attemptId == "0") { - return newBaseURI + "/api/v1/applications/" + appId + "/" + "1" + "/stages/" + stageId; + return newBaseURI + "/api/v1/applications/" + appId + "/stages/" + stageId; } else { return newBaseURI + "/api/v1/applications/" + appId + "/" + attemptId + "/stages/" + stageId; } } - var stageIdLen = words[ind + 3].indexOf('&'); - var stageId = words[ind + 3].substr(9, stageIdLen - 9); + var stageIdLen = words2[1].indexOf('&'); + var stageId = words2[1].substr(3, stageIdLen - 3); return location.origin + "/api/v1/applications/" + appId + "/stages/" + stageId; } From 25b5215cd6dbbc11b5e5d8d56bff6743578de2b5 Mon Sep 17 00:00:00 2001 From: pgandhi Date: Fri, 6 Oct 2017 14:07:18 -0500 Subject: [PATCH 07/10] [SPARK-21809]: Made changes as per comments Made some js functions common, added link to completed tasks, changed the name of taskspages.js to stagepage.js etc. --- .../apache/spark/ui/static/executorspage.js | 23 +----------- .../ui/static/{taskspages.js => stagepage.js} | 37 +++++-------------- .../org/apache/spark/ui/static/utils.js | 23 ++++++++++++ .../status/api/v1/AllStagesResource.scala | 2 +- .../apache/spark/ui/exec/ExecutorsTab.scala | 2 + .../org/apache/spark/ui/jobs/StagePage.scala | 2 +- 6 files changed, 38 insertions(+), 51 deletions(-) rename core/src/main/resources/org/apache/spark/ui/static/{taskspages.js => stagepage.js} (95%) diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js index 4ac5622545b7e..8b4dbfbd4ddcb 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js @@ -53,21 +53,6 @@ $(document).ajaxStart(function () { $.blockUI({message: '

Loading Executors Page...

'}); }); -function createTemplateURI(appId) { - var words = document.baseURI.split('/'); - var ind = words.indexOf("proxy"); - if (ind > 0) { - var baseURI = words.slice(0, ind + 1).join('/') + '/' + appId + '/static/executorspage-template.html'; - return baseURI; - } - ind = words.indexOf("history"); - if(ind > 0) { - var baseURI = words.slice(0, ind).join('/') + '/static/executorspage-template.html'; - return baseURI; - } - return location.origin + "/static/executorspage-template.html"; -} - function createRESTEndPoint(appId) { var words = document.baseURI.split('/'); var ind = words.indexOf("proxy"); @@ -137,11 +122,7 @@ function totalDurationColor(totalGCTime, totalDuration) { } $(document).ready(function () { - $.extend($.fn.dataTable.defaults, { - stateSave: true, - lengthMenu: [[20, 40, 60, 100, -1], [20, 40, 60, 100, "All"]], - pageLength: 20 - }); + setDataTableDefaults(); executorsSummary = $("#active-executors"); @@ -367,7 +348,7 @@ $(document).ready(function () { }; var data = {executors: response, "execSummary": [activeSummary, deadSummary, totalSummary]}; - $.get(createTemplateURI(appId), function (template) { + $.get(createTemplateURI(appId, "executorspage"), function (template) { executorsSummary.append(Mustache.render($(template).filter("#executors-summary-template").html(), data)); var selector = "#active-executors-table"; diff --git a/core/src/main/resources/org/apache/spark/ui/static/taskspages.js b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js similarity index 95% rename from core/src/main/resources/org/apache/spark/ui/static/taskspages.js rename to core/src/main/resources/org/apache/spark/ui/static/stagepage.js index 29c2ef70fe1f7..5be88ba363efa 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/taskspages.js +++ b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js @@ -17,7 +17,7 @@ $(document).ajaxStop($.unblockUI); $(document).ajaxStart(function () { - $.blockUI({message: '

Loading Tasks Page...

'}); + $.blockUI({message: '

Loading Stage Page...

'}); }); $.extend( $.fn.dataTable.ext.type.order, { @@ -36,24 +36,9 @@ $.extend( $.fn.dataTable.ext.type.order, { } } ); -function createTemplateURI(appId) { - var words = document.baseURI.split('/'); - var ind = words.indexOf("proxy"); - if (ind > 0) { - var baseURI = words.slice(0, ind + 1).join('/') + '/' + appId + '/static/stagespage-template.html'; - return baseURI; - } - ind = words.indexOf("history"); - if(ind > 0) { - var baseURI = words.slice(0, ind).join('/') + '/static/stagespage-template.html'; - return baseURI; - } - return location.origin + "/static/stagespage-template.html"; -} - // This function will only parse the URL under certain formate // e.g. https://axonitered-jt1.red.ygrid.yahoo.com:50509/history/application_1502220952225_59143/stages/stage/?id=0&attempt=0 -function StageEndPoint(appId) { +function stageEndPoint(appId) { var words = document.baseURI.split('/'); var words2 = document.baseURI.split('?'); var ind = words.indexOf("proxy"); @@ -99,11 +84,7 @@ function quantile(array, percentile) { } $(document).ready(function () { - $.extend($.fn.dataTable.defaults, { - stateSave: true, - lengthMenu: [[20, 40, 60, 100, -1], [20, 40, 60, 100, "All"]], - pageLength: 20 - }); + setDataTableDefaults(); $("#showAdditionalMetrics").append( "
" + @@ -124,20 +105,20 @@ $(document).ready(function () { tasksSummary = $("#active-tasks"); getStandAloneAppId(function (appId) { - var endPoint = StageEndPoint(appId); + 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){ + indices.forEach(function (ix) { task_table.push(response[0].tasks[ix]); }); // prepare data for task aggregated metrics table indices = Object.keys(response[0].executorSummary); var task_summary_table = []; - indices.forEach(function (ix){ + indices.forEach(function (ix) { response[0].executorSummary[ix].id = ix; task_summary_table.push(response[0].executorSummary[ix]); }); @@ -190,7 +171,7 @@ $(document).ready(function () { var indices = Object.keys(response[0].accumulatorUpdates); var accumulator_table_all = []; var accumulator_table = []; - indices.forEach(function (ix){ + indices.forEach(function (ix) { accumulator_table_all.push(response[0].accumulatorUpdates[ix]); }); @@ -203,7 +184,7 @@ $(document).ready(function () { // rendering the UI page var data = {executors: response, "taskstable": task_table, "task_metrics_table": task_metrics_table}; - $.get(createTemplateURI(appId), function(template) { + $.get(createTemplateURI(appId, "stagespage"), function(template) { tasksSummary.append(Mustache.render($(template).filter("#stages-summary-template").html(), data)); $("#taskMetric").click(function(){ @@ -461,7 +442,7 @@ $(document).ready(function () { }); // title number and toggle list - $("#summaryMetricsTitle").html("Summary Metrics for " + task_table.length + " Completed Tasks"); + $("#summaryMetricsTitle").html("Summary Metrics for " + "" + task_table.length + " Completed Tasks" + ""); $("#tasksTitle").html("Task (" + task_table.length + ")"); // hide or show the accumulate update table 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 8431c0dca8223..0562a3077a5ee 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 @@ -107,3 +107,26 @@ function ConvertDurationString(data) { } 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 + }); +} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala index 05a94ac37f2a8..c4e4af9a8e94c 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala @@ -161,7 +161,7 @@ private[v1] object AllStagesResource { info.finishTime - info.gettingResultTime } else { // The task is still fetching the result. - System.currentTimeMillis() - info.gettingResultTime + currentTime - info.gettingResultTime } } else { 0L diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala index ddb9b7c40ff3e..ca163074d885c 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala @@ -74,7 +74,9 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener, conf: Spar private val retainedDeadExecutors = conf.getInt("spark.ui.retainedDeadExecutors", 100) def activeStorageStatusList: Seq[StorageStatus] = storageStatusListener.storageStatusList + def deadStorageStatusList: Seq[StorageStatus] = storageStatusListener.deadStorageStatusList + override def onExecutorAdded( executorAdded: SparkListenerExecutorAdded): Unit = synchronized { val eid = executorAdded.executorId 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 2eca83df33f19..a2278336bfd7d 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 @@ -268,7 +268,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { currentTime) ++
- +
UIUtils.headerSparkPage(stageHeader, content, parent, showVisualization = true, useDataTables = true) From 0b2a8cfaab8fa6bcb92176f74dce2f47ba65454d Mon Sep 17 00:00:00 2001 From: pgandhi Date: Mon, 9 Oct 2017 15:34:34 -0500 Subject: [PATCH 08/10] [SPARK-21809] : Fixed accumulators table issue --- .../apache/spark/ui/static/executorspage.js | 7 +++-- .../org/apache/spark/ui/static/stagepage.js | 26 +++++++++++++++---- .../spark/ui/static/stagespage-template.html | 6 +++-- .../org/apache/spark/ui/static/utils.js | 11 +++----- 4 files changed, 34 insertions(+), 16 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js index 8b4dbfbd4ddcb..3a9c2a291593d 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js @@ -126,7 +126,10 @@ $(document).ready(function () { executorsSummary = $("#active-executors"); - getStandAloneAppId(function (appId) { + var app_id = getStandAloneAppId(); + buildExecutorDataTables(app_id); + + function buildExecutorDataTables(appId) { var endPoint = createRESTEndPoint(appId); $.getJSON(endPoint, function (response, status, jqXHR) { @@ -545,5 +548,5 @@ $(document).ready(function () { }); }); - }); + } }); 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 5be88ba363efa..e0abbc6569331 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 @@ -103,7 +103,10 @@ $(document).ready(function () { "
"); tasksSummary = $("#active-tasks"); - getStandAloneAppId(function (appId) { + var app_id = getStandAloneAppId(); + buildStageDataTables(app_id); + + function buildStageDataTables(appId) { var endPoint = stageEndPoint(appId); $.getJSON(endPoint, function(response, status, jqXHR) { @@ -381,6 +384,15 @@ $(document).ready(function () { return type === 'display' ? formatBytes(row.taskMetrics.peakExecutionMemory, type) : row.taskMetrics.peakExecutionMemory; } }, + { + data : function (row, type) { + if (accumulator_table.length > 0) { + return row.accumulatorUpdates[0].name + ' : ' + row.accumulatorUpdates[0].update; + } else { + return 0; + } + } + }, { data : function (row, type) { var msg = row.errorMessage; @@ -402,7 +414,8 @@ $(document).ready(function () { { "visible": false, "targets": 13 }, { "visible": false, "targets": 14 }, { "visible": false, "targets": 15 }, - { "visible": false, "targets": 16 } + { "visible": false, "targets": 16 }, + { "visible": false, "targets": 17 } ], "order": [[0, "asc"]] }; @@ -446,10 +459,13 @@ $(document).ready(function () { $("#tasksTitle").html("Task (" + task_table.length + ")"); // hide or show the accumulate update table - if(accumulator_table.length == 0){ - $("accumulator-update-table").hide(); + if (accumulator_table.length == 0) { + $("#accumulator-update-table").hide(); + } else { + taskTableSelector.column(17).visible(true); + $("#accumulator-update-table").show(); } }); }); - }); + } }); 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 aeb27f3600b0a..1df5765744d6b 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 @@ -67,9 +67,10 @@

Aggregated Metrics by Executor

-