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 5c91304e49fd..16806d9af909 100644
--- a/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html
+++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html
@@ -16,7 +16,7 @@
-->
diff --git a/core/src/main/resources/org/apache/spark/ui/static/utils.js b/core/src/main/resources/org/apache/spark/ui/static/utils.js
index edc0ee2ce181..9d3bc7998152 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,92 @@ 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 += '
'
+ });
+ return result;
+}
+
+function getStandAloneAppId(cb) {
+ var words = document.baseURI.split('/');
+ var ind = words.indexOf("proxy");
+ if (ind > 0) {
+ var appId = words[ind + 1];
+ cb(appId);
+ return;
+ }
+ ind = words.indexOf("history");
+ if (ind > 0) {
+ var appId = words[ind + 1];
+ cb(appId);
+ return;
+ }
+ //Looks like Web UI is running in standalone mode
+ //Let's get application-id using REST End Point
+ $.getJSON(location.origin + "/api/v1/applications", function(response, status, jqXHR) {
+ if (response && response.length > 0) {
+ var appId = response[0].id;
+ cb(appId);
+ return;
+ }
+ });
+}
+
+// This function is a helper function for sorting in datatable.
+// When the data is in duration (e.g. 12ms 2s 2min 2h )
+// It will convert the string into integer for correct ordering
+function ConvertDurationString(data) {
+ data = data.toString();
+ var units = data.replace(/[\d\.]/g, '' )
+ .replace(' ', '')
+ .toLowerCase();
+ var multiplier = 1;
+
+ switch(units) {
+ case 's':
+ multiplier = 1000;
+ break;
+ case 'min':
+ multiplier = 600000;
+ break;
+ case 'h':
+ multiplier = 3600000;
+ break;
+ default:
+ break;
+ }
+ return parseFloat(data) * multiplier;
+}
+
+function createTemplateURI(appId, templateName) {
+ var words = document.baseURI.split('/');
+ var ind = words.indexOf("proxy");
+ if (ind > 0) {
+ var baseURI = words.slice(0, ind + 1).join('/') + '/' + appId + '/static/' + templateName + '-template.html';
+ return baseURI;
+ }
+ ind = words.indexOf("history");
+ if(ind > 0) {
+ var baseURI = words.slice(0, ind).join('/') + '/static/' + templateName + '-template.html';
+ return baseURI;
+ }
+ return location.origin + "/static/" + templateName + "-template.html";
+}
+
+function setDataTableDefaults() {
+ $.extend($.fn.dataTable.defaults, {
+ stateSave: true,
+ lengthMenu: [[20, 40, 60, 100, -1], [20, 40, 60, 100, "All"]],
+ pageLength: 20
+ });
+}
+
+function formatDate(date) {
+ if (date <= 0) return "-";
+ else return date.split(".")[0].replace("T", " ");
+}
diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css
index 935d9b1aec61..5ed5886a7b70 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 4a4ed954d689..c4e4af9a8e94 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,8 +20,9 @@ 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.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
@@ -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.
+ currentTime - 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 f15073bccced..479da4d36b2f 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 05948f266105..b0980255a141 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
@@ -53,8 +53,16 @@ class ExecutorStageSummary private[spark](
val outputBytes : Long,
val shuffleRead : Long,
val shuffleWrite : Long,
+ val shuffleWriteRecords : Long,
val memoryBytesSpilled : Long,
- val diskBytesSpilled : 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])
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 64a1a292a384..ca163074d885 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,6 +68,7 @@ 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)
@@ -170,6 +172,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 382a6f979f2e..000000000000
--- 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
- }
-
-
-
- | Executor ID |
- Address |
- Task Time |
- Total Tasks |
- Failed Tasks |
- Killed Tasks |
- Succeeded Tasks |
- {if (hasInput) {
-
- Input Size / Records
- |
- }}
- {if (hasOutput) {
-
- Output Size / Records
- |
- }}
- {if (hasShuffleRead) {
-
-
- Shuffle Read Size / Records
- |
- }}
- {if (hasShuffleWrite) {
-
-
- Shuffle Write Size / Records
- |
- }}
- {if (hasBytesSpilled) {
- Shuffle Spill (Memory) |
- Shuffle Spill (Disk) |
- }}
-
-
- Blacklisted
-
- |
-
-
- {createExecutorTable()}
-
-
-
- }
-
- 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) =>
- }
- }
-
- |
- {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 4d80308eb0a6..6393cbb58a3a 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 =
@@ -195,88 +193,9 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
- val showAdditionalMetrics =
-
-
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.
@@ -287,7 +206,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
}
}
val currentTime = System.currentTimeMillis()
- val (taskTable, taskTableHTML) = try {
+ val taskTable = try {
val _taskTable = new TaskPagedTable(
parent.conf,
UIUtils.prependBaseUri(parent.basePath) +
@@ -306,17 +225,11 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
desc = taskSortDesc,
executorsListener = executorsListener
)
- (_taskTable, _taskTable.table(page))
+ _taskTable.table(page)
+ _taskTable
} catch {
case e @ (_ : IllegalArgumentException | _ : IndexOutOfBoundsException) =>
- val errorMessage =
-
-
Error while rendering stage table:
-
- {Utils.exceptionString(e)}
-
-
- (null, errorMessage)
+ null
}
val jsForScrollingDownToTaskTable =
@@ -340,261 +253,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 =
-
-
-
-
- {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) ++
- ++
- {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 5acec0d0f54c..28b8c0a698d8 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/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json
index 6fb40f6f1713..c34f11986783 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 f5a89a210764..0ba8c185b263 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 9b401b414f8d..25e9296e0da4 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 2ebee66a6d7c..b53458e313c5 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 965a31a4104c..d428289d0922 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 31132e156937..ff3d2a3285cf 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 6af1cfbeb8f7..131a747d6cf0 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 6af1cfbeb8f7..131a747d6cf0 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 c26daf4b8d7b..f94d7106e072 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 44b5f66efe33..b105516d1bf7 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/status/api/v1/AllStagesResourceSuite.scala b/core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala
index 82bd7c4ff660..a833c40f9788 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
}
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 499d47b13d70..000000000000
--- 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 267c8dc1bd75..153111af3983 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")
diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala
index dd299e074535..1532a58a8eea 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"),