[SPARK-32125][UI] Support get taskList by status in Web UI and SHS Rest API

### What changes were proposed in this pull request?
Support fetching taskList by status as below:
```
/applications/[app-id]/stages/[stage-id]/[stage-attempt-id]/taskList?status=failed
```

### Why are the changes needed?

When there're large number of tasks in one stage, current api is hard to get taskList by status

### Does this PR introduce _any_ user-facing change?
Yes. Updated monitoring doc.

### How was this patch tested?
Added tests in `HistoryServerSuite`

Closes #28942 from warrenzhu25/SPARK-32125.

Authored-by: Warren Zhu <zhonzh@microsoft.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
This commit is contained in:
Warren Zhu 2020-07-16 11:31:24 +08:00 committed by Gengliang Wang
parent c28a6fa511
commit db47c6e340
8 changed files with 1666 additions and 7 deletions

View file

@ -0,0 +1,32 @@
/*
* 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.status.api.v1;
import org.apache.spark.util.EnumUtil;
public enum TaskStatus {
RUNNING,
KILLED,
FAILED,
SUCCESS,
UNKNOWN;
public static TaskStatus fromString(String str) {
return EnumUtil.parseIgnoreCase(TaskStatus.class, str);
}
}

View file

@ -386,7 +386,8 @@ private[spark] class AppStatusStore(
stageAttemptId: Int,
offset: Int,
length: Int,
sortBy: v1.TaskSorting): Seq[v1.TaskData] = {
sortBy: v1.TaskSorting,
statuses: JList[v1.TaskStatus]): Seq[v1.TaskData] = {
val (indexName, ascending) = sortBy match {
case v1.TaskSorting.ID =>
(None, true)
@ -395,7 +396,7 @@ private[spark] class AppStatusStore(
case v1.TaskSorting.DECREASING_RUNTIME =>
(Some(TaskIndexNames.EXEC_RUN_TIME), false)
}
taskList(stageId, stageAttemptId, offset, length, indexName, ascending)
taskList(stageId, stageAttemptId, offset, length, indexName, ascending, statuses)
}
def taskList(
@ -404,7 +405,8 @@ private[spark] class AppStatusStore(
offset: Int,
length: Int,
sortBy: Option[String],
ascending: Boolean): Seq[v1.TaskData] = {
ascending: Boolean,
statuses: JList[v1.TaskStatus] = List().asJava): Seq[v1.TaskData] = {
val stageKey = Array(stageId, stageAttemptId)
val base = store.view(classOf[TaskDataWrapper])
val indexed = sortBy match {
@ -417,7 +419,13 @@ private[spark] class AppStatusStore(
}
val ordered = if (ascending) indexed else indexed.reverse()
val taskDataWrapperIter = ordered.skip(offset).max(length).asScala
val taskDataWrapperIter = if (statuses != null && !statuses.isEmpty) {
val statusesStr = statuses.asScala.map(_.toString).toSet
ordered.asScala.filter(s => statusesStr.contains(s.status)).slice(offset, offset + length)
} else {
ordered.skip(offset).max(length).asScala
}
constructTaskDataList(taskDataWrapperIter)
}

View file

@ -96,8 +96,9 @@ private[v1] class StagesResource extends BaseAppResource {
@PathParam("stageAttemptId") stageAttemptId: Int,
@DefaultValue("0") @QueryParam("offset") offset: Int,
@DefaultValue("20") @QueryParam("length") length: Int,
@DefaultValue("ID") @QueryParam("sortBy") sortBy: TaskSorting): Seq[TaskData] = {
withUI(_.store.taskList(stageId, stageAttemptId, offset, length, sortBy))
@DefaultValue("ID") @QueryParam("sortBy") sortBy: TaskSorting,
@QueryParam("status") statuses: JList[TaskStatus]): Seq[TaskData] = {
withUI(_.store.taskList(stageId, stageAttemptId, offset, length, sortBy, statuses))
}
// This api needs to stay formatted exactly as it is below, since, it is being used by the

View file

@ -0,0 +1,99 @@
[ {
"taskId" : 1,
"index" : 1,
"attempt" : 0,
"launchTime" : "2015-05-06T13:03:06.502GMT",
"duration" : 421,
"executorId" : "driver",
"host" : "localhost",
"status" : "SUCCESS",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"taskMetrics" : {
"executorDeserializeTime" : 31,
"executorDeserializeCpuTime" : 0,
"executorRunTime" : 350,
"executorCpuTime" : 0,
"resultSize" : 2010,
"jvmGcTime" : 7,
"resultSerializationTime" : 0,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"peakExecutionMemory" : 0,
"inputMetrics" : {
"bytesRead" : 60488,
"recordsRead" : 10000
},
"outputMetrics" : {
"bytesWritten" : 0,
"recordsWritten" : 0
},
"shuffleReadMetrics" : {
"remoteBlocksFetched" : 0,
"localBlocksFetched" : 0,
"fetchWaitTime" : 0,
"remoteBytesRead" : 0,
"remoteBytesReadToDisk" : 0,
"localBytesRead" : 0,
"recordsRead" : 0
},
"shuffleWriteMetrics" : {
"bytesWritten" : 1710,
"writeTime" : 3934399,
"recordsWritten" : 10
}
},
"executorLogs" : { },
"schedulerDelay" : 40,
"gettingResultTime" : 0
}, {
"taskId" : 2,
"index" : 2,
"attempt" : 0,
"launchTime" : "2015-05-06T13:03:06.503GMT",
"duration" : 419,
"executorId" : "driver",
"host" : "localhost",
"status" : "SUCCESS",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"taskMetrics" : {
"executorDeserializeTime" : 32,
"executorDeserializeCpuTime" : 0,
"executorRunTime" : 348,
"executorCpuTime" : 0,
"resultSize" : 2010,
"jvmGcTime" : 7,
"resultSerializationTime" : 2,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"peakExecutionMemory" : 0,
"inputMetrics" : {
"bytesRead" : 60488,
"recordsRead" : 10000
},
"outputMetrics" : {
"bytesWritten" : 0,
"recordsWritten" : 0
},
"shuffleReadMetrics" : {
"remoteBlocksFetched" : 0,
"localBlocksFetched" : 0,
"fetchWaitTime" : 0,
"remoteBytesRead" : 0,
"remoteBytesReadToDisk" : 0,
"localBytesRead" : 0,
"recordsRead" : 0
},
"shuffleWriteMetrics" : {
"bytesWritten" : 1710,
"writeTime" : 89885,
"recordsWritten" : 10
}
},
"executorLogs" : { },
"schedulerDelay" : 37,
"gettingResultTime" : 0
} ]

View file

@ -0,0 +1,981 @@
[ {
"taskId" : 40,
"index" : 40,
"attempt" : 0,
"launchTime" : "2015-05-06T13:03:07.197GMT",
"duration" : 24,
"executorId" : "driver",
"host" : "localhost",
"status" : "SUCCESS",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"taskMetrics" : {
"executorDeserializeTime" : 4,
"executorDeserializeCpuTime" : 0,
"executorRunTime" : 14,
"executorCpuTime" : 0,
"resultSize" : 2065,
"jvmGcTime" : 0,
"resultSerializationTime" : 0,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"peakExecutionMemory" : 0,
"inputMetrics" : {
"bytesRead" : 70564,
"recordsRead" : 10000
},
"outputMetrics" : {
"bytesWritten" : 0,
"recordsWritten" : 0
},
"shuffleReadMetrics" : {
"remoteBlocksFetched" : 0,
"localBlocksFetched" : 0,
"fetchWaitTime" : 0,
"remoteBytesRead" : 0,
"remoteBytesReadToDisk" : 0,
"localBytesRead" : 0,
"recordsRead" : 0
},
"shuffleWriteMetrics" : {
"bytesWritten" : 1710,
"writeTime" : 94792,
"recordsWritten" : 10
}
},
"executorLogs" : { },
"schedulerDelay" : 6,
"gettingResultTime" : 0
}, {
"taskId" : 41,
"index" : 41,
"attempt" : 0,
"launchTime" : "2015-05-06T13:03:07.200GMT",
"duration" : 24,
"executorId" : "driver",
"host" : "localhost",
"status" : "SUCCESS",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"taskMetrics" : {
"executorDeserializeTime" : 2,
"executorDeserializeCpuTime" : 0,
"executorRunTime" : 16,
"executorCpuTime" : 0,
"resultSize" : 2065,
"jvmGcTime" : 0,
"resultSerializationTime" : 0,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"peakExecutionMemory" : 0,
"inputMetrics" : {
"bytesRead" : 70564,
"recordsRead" : 10000
},
"outputMetrics" : {
"bytesWritten" : 0,
"recordsWritten" : 0
},
"shuffleReadMetrics" : {
"remoteBlocksFetched" : 0,
"localBlocksFetched" : 0,
"fetchWaitTime" : 0,
"remoteBytesRead" : 0,
"remoteBytesReadToDisk" : 0,
"localBytesRead" : 0,
"recordsRead" : 0
},
"shuffleWriteMetrics" : {
"bytesWritten" : 1710,
"writeTime" : 90765,
"recordsWritten" : 10
}
},
"executorLogs" : { },
"schedulerDelay" : 6,
"gettingResultTime" : 0
}, {
"taskId" : 43,
"index" : 43,
"attempt" : 0,
"launchTime" : "2015-05-06T13:03:07.204GMT",
"duration" : 39,
"executorId" : "driver",
"host" : "localhost",
"status" : "SUCCESS",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"taskMetrics" : {
"executorDeserializeTime" : 2,
"executorDeserializeCpuTime" : 0,
"executorRunTime" : 16,
"executorCpuTime" : 0,
"resultSize" : 2065,
"jvmGcTime" : 0,
"resultSerializationTime" : 0,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"peakExecutionMemory" : 0,
"inputMetrics" : {
"bytesRead" : 70564,
"recordsRead" : 10000
},
"outputMetrics" : {
"bytesWritten" : 0,
"recordsWritten" : 0
},
"shuffleReadMetrics" : {
"remoteBlocksFetched" : 0,
"localBlocksFetched" : 0,
"fetchWaitTime" : 0,
"remoteBytesRead" : 0,
"remoteBytesReadToDisk" : 0,
"localBytesRead" : 0,
"recordsRead" : 0
},
"shuffleWriteMetrics" : {
"bytesWritten" : 1710,
"writeTime" : 171516,
"recordsWritten" : 10
}
},
"executorLogs" : { },
"schedulerDelay" : 21,
"gettingResultTime" : 0
}, {
"taskId" : 57,
"index" : 57,
"attempt" : 0,
"launchTime" : "2015-05-06T13:03:07.257GMT",
"duration" : 21,
"executorId" : "driver",
"host" : "localhost",
"status" : "SUCCESS",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"taskMetrics" : {
"executorDeserializeTime" : 3,
"executorDeserializeCpuTime" : 0,
"executorRunTime" : 16,
"executorCpuTime" : 0,
"resultSize" : 2065,
"jvmGcTime" : 0,
"resultSerializationTime" : 0,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"peakExecutionMemory" : 0,
"inputMetrics" : {
"bytesRead" : 70564,
"recordsRead" : 10000
},
"outputMetrics" : {
"bytesWritten" : 0,
"recordsWritten" : 0
},
"shuffleReadMetrics" : {
"remoteBlocksFetched" : 0,
"localBlocksFetched" : 0,
"fetchWaitTime" : 0,
"remoteBytesRead" : 0,
"remoteBytesReadToDisk" : 0,
"localBytesRead" : 0,
"recordsRead" : 0
},
"shuffleWriteMetrics" : {
"bytesWritten" : 1710,
"writeTime" : 96849,
"recordsWritten" : 10
}
},
"executorLogs" : { },
"schedulerDelay" : 2,
"gettingResultTime" : 0
}, {
"taskId" : 58,
"index" : 58,
"attempt" : 0,
"launchTime" : "2015-05-06T13:03:07.263GMT",
"duration" : 23,
"executorId" : "driver",
"host" : "localhost",
"status" : "SUCCESS",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"taskMetrics" : {
"executorDeserializeTime" : 3,
"executorDeserializeCpuTime" : 0,
"executorRunTime" : 16,
"executorCpuTime" : 0,
"resultSize" : 2065,
"jvmGcTime" : 0,
"resultSerializationTime" : 0,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"peakExecutionMemory" : 0,
"inputMetrics" : {
"bytesRead" : 70564,
"recordsRead" : 10000
},
"outputMetrics" : {
"bytesWritten" : 0,
"recordsWritten" : 0
},
"shuffleReadMetrics" : {
"remoteBlocksFetched" : 0,
"localBlocksFetched" : 0,
"fetchWaitTime" : 0,
"remoteBytesRead" : 0,
"remoteBytesReadToDisk" : 0,
"localBytesRead" : 0,
"recordsRead" : 0
},
"shuffleWriteMetrics" : {
"bytesWritten" : 1710,
"writeTime" : 97521,
"recordsWritten" : 10
}
},
"executorLogs" : { },
"schedulerDelay" : 4,
"gettingResultTime" : 0
}, {
"taskId" : 68,
"index" : 68,
"attempt" : 0,
"launchTime" : "2015-05-06T13:03:07.306GMT",
"duration" : 22,
"executorId" : "driver",
"host" : "localhost",
"status" : "SUCCESS",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"taskMetrics" : {
"executorDeserializeTime" : 2,
"executorDeserializeCpuTime" : 0,
"executorRunTime" : 16,
"executorCpuTime" : 0,
"resultSize" : 2065,
"jvmGcTime" : 0,
"resultSerializationTime" : 0,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"peakExecutionMemory" : 0,
"inputMetrics" : {
"bytesRead" : 70564,
"recordsRead" : 10000
},
"outputMetrics" : {
"bytesWritten" : 0,
"recordsWritten" : 0
},
"shuffleReadMetrics" : {
"remoteBlocksFetched" : 0,
"localBlocksFetched" : 0,
"fetchWaitTime" : 0,
"remoteBytesRead" : 0,
"remoteBytesReadToDisk" : 0,
"localBytesRead" : 0,
"recordsRead" : 0
},
"shuffleWriteMetrics" : {
"bytesWritten" : 1710,
"writeTime" : 101750,
"recordsWritten" : 10
}
},
"executorLogs" : { },
"schedulerDelay" : 4,
"gettingResultTime" : 0
}, {
"taskId" : 86,
"index" : 86,
"attempt" : 0,
"launchTime" : "2015-05-06T13:03:07.374GMT",
"duration" : 28,
"executorId" : "driver",
"host" : "localhost",
"status" : "SUCCESS",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"taskMetrics" : {
"executorDeserializeTime" : 3,
"executorDeserializeCpuTime" : 0,
"executorRunTime" : 16,
"executorCpuTime" : 0,
"resultSize" : 2065,
"jvmGcTime" : 0,
"resultSerializationTime" : 1,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"peakExecutionMemory" : 0,
"inputMetrics" : {
"bytesRead" : 70564,
"recordsRead" : 10000
},
"outputMetrics" : {
"bytesWritten" : 0,
"recordsWritten" : 0
},
"shuffleReadMetrics" : {
"remoteBlocksFetched" : 0,
"localBlocksFetched" : 0,
"fetchWaitTime" : 0,
"remoteBytesRead" : 0,
"remoteBytesReadToDisk" : 0,
"localBytesRead" : 0,
"recordsRead" : 0
},
"shuffleWriteMetrics" : {
"bytesWritten" : 1710,
"writeTime" : 95848,
"recordsWritten" : 10
}
},
"executorLogs" : { },
"schedulerDelay" : 8,
"gettingResultTime" : 0
}, {
"taskId" : 32,
"index" : 32,
"attempt" : 0,
"launchTime" : "2015-05-06T13:03:07.148GMT",
"duration" : 33,
"executorId" : "driver",
"host" : "localhost",
"status" : "SUCCESS",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"taskMetrics" : {
"executorDeserializeTime" : 3,
"executorDeserializeCpuTime" : 0,
"executorRunTime" : 17,
"executorCpuTime" : 0,
"resultSize" : 2065,
"jvmGcTime" : 0,
"resultSerializationTime" : 0,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"peakExecutionMemory" : 0,
"inputMetrics" : {
"bytesRead" : 70564,
"recordsRead" : 10000
},
"outputMetrics" : {
"bytesWritten" : 0,
"recordsWritten" : 0
},
"shuffleReadMetrics" : {
"remoteBlocksFetched" : 0,
"localBlocksFetched" : 0,
"fetchWaitTime" : 0,
"remoteBytesRead" : 0,
"remoteBytesReadToDisk" : 0,
"localBytesRead" : 0,
"recordsRead" : 0
},
"shuffleWriteMetrics" : {
"bytesWritten" : 1710,
"writeTime" : 89603,
"recordsWritten" : 10
}
},
"executorLogs" : { },
"schedulerDelay" : 13,
"gettingResultTime" : 0
}, {
"taskId" : 39,
"index" : 39,
"attempt" : 0,
"launchTime" : "2015-05-06T13:03:07.180GMT",
"duration" : 32,
"executorId" : "driver",
"host" : "localhost",
"status" : "SUCCESS",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"taskMetrics" : {
"executorDeserializeTime" : 2,
"executorDeserializeCpuTime" : 0,
"executorRunTime" : 17,
"executorCpuTime" : 0,
"resultSize" : 2065,
"jvmGcTime" : 0,
"resultSerializationTime" : 0,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"peakExecutionMemory" : 0,
"inputMetrics" : {
"bytesRead" : 70564,
"recordsRead" : 10000
},
"outputMetrics" : {
"bytesWritten" : 0,
"recordsWritten" : 0
},
"shuffleReadMetrics" : {
"remoteBlocksFetched" : 0,
"localBlocksFetched" : 0,
"fetchWaitTime" : 0,
"remoteBytesRead" : 0,
"remoteBytesReadToDisk" : 0,
"localBytesRead" : 0,
"recordsRead" : 0
},
"shuffleWriteMetrics" : {
"bytesWritten" : 1710,
"writeTime" : 98748,
"recordsWritten" : 10
}
},
"executorLogs" : { },
"schedulerDelay" : 13,
"gettingResultTime" : 0
}, {
"taskId" : 42,
"index" : 42,
"attempt" : 0,
"launchTime" : "2015-05-06T13:03:07.203GMT",
"duration" : 42,
"executorId" : "driver",
"host" : "localhost",
"status" : "SUCCESS",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"taskMetrics" : {
"executorDeserializeTime" : 10,
"executorDeserializeCpuTime" : 0,
"executorRunTime" : 17,
"executorCpuTime" : 0,
"resultSize" : 2065,
"jvmGcTime" : 0,
"resultSerializationTime" : 0,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"peakExecutionMemory" : 0,
"inputMetrics" : {
"bytesRead" : 70564,
"recordsRead" : 10000
},
"outputMetrics" : {
"bytesWritten" : 0,
"recordsWritten" : 0
},
"shuffleReadMetrics" : {
"remoteBlocksFetched" : 0,
"localBlocksFetched" : 0,
"fetchWaitTime" : 0,
"remoteBytesRead" : 0,
"remoteBytesReadToDisk" : 0,
"localBytesRead" : 0,
"recordsRead" : 0
},
"shuffleWriteMetrics" : {
"bytesWritten" : 1710,
"writeTime" : 103713,
"recordsWritten" : 10
}
},
"executorLogs" : { },
"schedulerDelay" : 15,
"gettingResultTime" : 0
}, {
"taskId" : 51,
"index" : 51,
"attempt" : 0,
"launchTime" : "2015-05-06T13:03:07.242GMT",
"duration" : 21,
"executorId" : "driver",
"host" : "localhost",
"status" : "SUCCESS",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"taskMetrics" : {
"executorDeserializeTime" : 2,
"executorDeserializeCpuTime" : 0,
"executorRunTime" : 17,
"executorCpuTime" : 0,
"resultSize" : 2065,
"jvmGcTime" : 0,
"resultSerializationTime" : 0,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"peakExecutionMemory" : 0,
"inputMetrics" : {
"bytesRead" : 70564,
"recordsRead" : 10000
},
"outputMetrics" : {
"bytesWritten" : 0,
"recordsWritten" : 0
},
"shuffleReadMetrics" : {
"remoteBlocksFetched" : 0,
"localBlocksFetched" : 0,
"fetchWaitTime" : 0,
"remoteBytesRead" : 0,
"remoteBytesReadToDisk" : 0,
"localBytesRead" : 0,
"recordsRead" : 0
},
"shuffleWriteMetrics" : {
"bytesWritten" : 1710,
"writeTime" : 96013,
"recordsWritten" : 10
}
},
"executorLogs" : { },
"schedulerDelay" : 2,
"gettingResultTime" : 0
}, {
"taskId" : 59,
"index" : 59,
"attempt" : 0,
"launchTime" : "2015-05-06T13:03:07.265GMT",
"duration" : 23,
"executorId" : "driver",
"host" : "localhost",
"status" : "SUCCESS",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"taskMetrics" : {
"executorDeserializeTime" : 3,
"executorDeserializeCpuTime" : 0,
"executorRunTime" : 17,
"executorCpuTime" : 0,
"resultSize" : 2065,
"jvmGcTime" : 0,
"resultSerializationTime" : 0,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"peakExecutionMemory" : 0,
"inputMetrics" : {
"bytesRead" : 70564,
"recordsRead" : 10000
},
"outputMetrics" : {
"bytesWritten" : 0,
"recordsWritten" : 0
},
"shuffleReadMetrics" : {
"remoteBlocksFetched" : 0,
"localBlocksFetched" : 0,
"fetchWaitTime" : 0,
"remoteBytesRead" : 0,
"remoteBytesReadToDisk" : 0,
"localBytesRead" : 0,
"recordsRead" : 0
},
"shuffleWriteMetrics" : {
"bytesWritten" : 1710,
"writeTime" : 100753,
"recordsWritten" : 10
}
},
"executorLogs" : { },
"schedulerDelay" : 3,
"gettingResultTime" : 0
}, {
"taskId" : 63,
"index" : 63,
"attempt" : 0,
"launchTime" : "2015-05-06T13:03:07.276GMT",
"duration" : 40,
"executorId" : "driver",
"host" : "localhost",
"status" : "SUCCESS",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"taskMetrics" : {
"executorDeserializeTime" : 20,
"executorDeserializeCpuTime" : 0,
"executorRunTime" : 17,
"executorCpuTime" : 0,
"resultSize" : 2065,
"jvmGcTime" : 5,
"resultSerializationTime" : 0,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"peakExecutionMemory" : 0,
"inputMetrics" : {
"bytesRead" : 70564,
"recordsRead" : 10000
},
"outputMetrics" : {
"bytesWritten" : 0,
"recordsWritten" : 0
},
"shuffleReadMetrics" : {
"remoteBlocksFetched" : 0,
"localBlocksFetched" : 0,
"fetchWaitTime" : 0,
"remoteBytesRead" : 0,
"remoteBytesReadToDisk" : 0,
"localBytesRead" : 0,
"recordsRead" : 0
},
"shuffleWriteMetrics" : {
"bytesWritten" : 1710,
"writeTime" : 102779,
"recordsWritten" : 10
}
},
"executorLogs" : { },
"schedulerDelay" : 3,
"gettingResultTime" : 0
}, {
"taskId" : 87,
"index" : 87,
"attempt" : 0,
"launchTime" : "2015-05-06T13:03:07.374GMT",
"duration" : 36,
"executorId" : "driver",
"host" : "localhost",
"status" : "SUCCESS",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"taskMetrics" : {
"executorDeserializeTime" : 12,
"executorDeserializeCpuTime" : 0,
"executorRunTime" : 17,
"executorCpuTime" : 0,
"resultSize" : 2065,
"jvmGcTime" : 0,
"resultSerializationTime" : 0,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"peakExecutionMemory" : 0,
"inputMetrics" : {
"bytesRead" : 70564,
"recordsRead" : 10000
},
"outputMetrics" : {
"bytesWritten" : 0,
"recordsWritten" : 0
},
"shuffleReadMetrics" : {
"remoteBlocksFetched" : 0,
"localBlocksFetched" : 0,
"fetchWaitTime" : 0,
"remoteBytesRead" : 0,
"remoteBytesReadToDisk" : 0,
"localBytesRead" : 0,
"recordsRead" : 0
},
"shuffleWriteMetrics" : {
"bytesWritten" : 1710,
"writeTime" : 102159,
"recordsWritten" : 10
}
},
"executorLogs" : { },
"schedulerDelay" : 7,
"gettingResultTime" : 0
}, {
"taskId" : 90,
"index" : 90,
"attempt" : 0,
"launchTime" : "2015-05-06T13:03:07.385GMT",
"duration" : 23,
"executorId" : "driver",
"host" : "localhost",
"status" : "SUCCESS",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"taskMetrics" : {
"executorDeserializeTime" : 2,
"executorDeserializeCpuTime" : 0,
"executorRunTime" : 17,
"executorCpuTime" : 0,
"resultSize" : 2065,
"jvmGcTime" : 0,
"resultSerializationTime" : 0,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"peakExecutionMemory" : 0,
"inputMetrics" : {
"bytesRead" : 70564,
"recordsRead" : 10000
},
"outputMetrics" : {
"bytesWritten" : 0,
"recordsWritten" : 0
},
"shuffleReadMetrics" : {
"remoteBlocksFetched" : 0,
"localBlocksFetched" : 0,
"fetchWaitTime" : 0,
"remoteBytesRead" : 0,
"remoteBytesReadToDisk" : 0,
"localBytesRead" : 0,
"recordsRead" : 0
},
"shuffleWriteMetrics" : {
"bytesWritten" : 1710,
"writeTime" : 98472,
"recordsWritten" : 10
}
},
"executorLogs" : { },
"schedulerDelay" : 4,
"gettingResultTime" : 0
}, {
"taskId" : 99,
"index" : 99,
"attempt" : 0,
"launchTime" : "2015-05-06T13:03:07.426GMT",
"duration" : 22,
"executorId" : "driver",
"host" : "localhost",
"status" : "SUCCESS",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"taskMetrics" : {
"executorDeserializeTime" : 2,
"executorDeserializeCpuTime" : 0,
"executorRunTime" : 17,
"executorCpuTime" : 0,
"resultSize" : 2065,
"jvmGcTime" : 0,
"resultSerializationTime" : 0,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"peakExecutionMemory" : 0,
"inputMetrics" : {
"bytesRead" : 70565,
"recordsRead" : 10000
},
"outputMetrics" : {
"bytesWritten" : 0,
"recordsWritten" : 0
},
"shuffleReadMetrics" : {
"remoteBlocksFetched" : 0,
"localBlocksFetched" : 0,
"fetchWaitTime" : 0,
"remoteBytesRead" : 0,
"remoteBytesReadToDisk" : 0,
"localBytesRead" : 0,
"recordsRead" : 0
},
"shuffleWriteMetrics" : {
"bytesWritten" : 1710,
"writeTime" : 133964,
"recordsWritten" : 10
}
},
"executorLogs" : { },
"schedulerDelay" : 3,
"gettingResultTime" : 0
}, {
"taskId" : 44,
"index" : 44,
"attempt" : 0,
"launchTime" : "2015-05-06T13:03:07.205GMT",
"duration" : 37,
"executorId" : "driver",
"host" : "localhost",
"status" : "SUCCESS",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"taskMetrics" : {
"executorDeserializeTime" : 3,
"executorDeserializeCpuTime" : 0,
"executorRunTime" : 18,
"executorCpuTime" : 0,
"resultSize" : 2065,
"jvmGcTime" : 0,
"resultSerializationTime" : 0,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"peakExecutionMemory" : 0,
"inputMetrics" : {
"bytesRead" : 70564,
"recordsRead" : 10000
},
"outputMetrics" : {
"bytesWritten" : 0,
"recordsWritten" : 0
},
"shuffleReadMetrics" : {
"remoteBlocksFetched" : 0,
"localBlocksFetched" : 0,
"fetchWaitTime" : 0,
"remoteBytesRead" : 0,
"remoteBytesReadToDisk" : 0,
"localBytesRead" : 0,
"recordsRead" : 0
},
"shuffleWriteMetrics" : {
"bytesWritten" : 1710,
"writeTime" : 98293,
"recordsWritten" : 10
}
},
"executorLogs" : { },
"schedulerDelay" : 16,
"gettingResultTime" : 0
}, {
"taskId" : 47,
"index" : 47,
"attempt" : 0,
"launchTime" : "2015-05-06T13:03:07.212GMT",
"duration" : 33,
"executorId" : "driver",
"host" : "localhost",
"status" : "SUCCESS",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"taskMetrics" : {
"executorDeserializeTime" : 2,
"executorDeserializeCpuTime" : 0,
"executorRunTime" : 18,
"executorCpuTime" : 0,
"resultSize" : 2065,
"jvmGcTime" : 0,
"resultSerializationTime" : 0,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"peakExecutionMemory" : 0,
"inputMetrics" : {
"bytesRead" : 70564,
"recordsRead" : 10000
},
"outputMetrics" : {
"bytesWritten" : 0,
"recordsWritten" : 0
},
"shuffleReadMetrics" : {
"remoteBlocksFetched" : 0,
"localBlocksFetched" : 0,
"fetchWaitTime" : 0,
"remoteBytesRead" : 0,
"remoteBytesReadToDisk" : 0,
"localBytesRead" : 0,
"recordsRead" : 0
},
"shuffleWriteMetrics" : {
"bytesWritten" : 1710,
"writeTime" : 103015,
"recordsWritten" : 10
}
},
"executorLogs" : { },
"schedulerDelay" : 13,
"gettingResultTime" : 0
}, {
"taskId" : 50,
"index" : 50,
"attempt" : 0,
"launchTime" : "2015-05-06T13:03:07.240GMT",
"duration" : 26,
"executorId" : "driver",
"host" : "localhost",
"status" : "SUCCESS",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"taskMetrics" : {
"executorDeserializeTime" : 4,
"executorDeserializeCpuTime" : 0,
"executorRunTime" : 18,
"executorCpuTime" : 0,
"resultSize" : 2065,
"jvmGcTime" : 0,
"resultSerializationTime" : 0,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"peakExecutionMemory" : 0,
"inputMetrics" : {
"bytesRead" : 70564,
"recordsRead" : 10000
},
"outputMetrics" : {
"bytesWritten" : 0,
"recordsWritten" : 0
},
"shuffleReadMetrics" : {
"remoteBlocksFetched" : 0,
"localBlocksFetched" : 0,
"fetchWaitTime" : 0,
"remoteBytesRead" : 0,
"remoteBytesReadToDisk" : 0,
"localBytesRead" : 0,
"recordsRead" : 0
},
"shuffleWriteMetrics" : {
"bytesWritten" : 1710,
"writeTime" : 90836,
"recordsWritten" : 10
}
},
"executorLogs" : { },
"schedulerDelay" : 4,
"gettingResultTime" : 0
}, {
"taskId" : 52,
"index" : 52,
"attempt" : 0,
"launchTime" : "2015-05-06T13:03:07.243GMT",
"duration" : 28,
"executorId" : "driver",
"host" : "localhost",
"status" : "SUCCESS",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"taskMetrics" : {
"executorDeserializeTime" : 5,
"executorDeserializeCpuTime" : 0,
"executorRunTime" : 18,
"executorCpuTime" : 0,
"resultSize" : 2065,
"jvmGcTime" : 0,
"resultSerializationTime" : 0,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"peakExecutionMemory" : 0,
"inputMetrics" : {
"bytesRead" : 70564,
"recordsRead" : 10000
},
"outputMetrics" : {
"bytesWritten" : 0,
"recordsWritten" : 0
},
"shuffleReadMetrics" : {
"remoteBlocksFetched" : 0,
"localBlocksFetched" : 0,
"fetchWaitTime" : 0,
"remoteBytesRead" : 0,
"remoteBytesReadToDisk" : 0,
"localBytesRead" : 0,
"recordsRead" : 0
},
"shuffleWriteMetrics" : {
"bytesWritten" : 1710,
"writeTime" : 89664,
"recordsWritten" : 10
}
},
"executorLogs" : { },
"schedulerDelay" : 5,
"gettingResultTime" : 0
} ]

View file

@ -0,0 +1,531 @@
[ {
"taskId" : 1,
"index" : 1,
"attempt" : 0,
"launchTime" : "2016-11-15T23:20:44.052GMT",
"duration" : 675,
"executorId" : "0",
"host" : "172.22.0.111",
"status" : "FAILED",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"errorMessage" : "java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n",
"taskMetrics" : {
"executorDeserializeTime" : 0,
"executorDeserializeCpuTime" : 0,
"executorRunTime" : 494,
"executorCpuTime" : 0,
"resultSize" : 0,
"jvmGcTime" : 30,
"resultSerializationTime" : 0,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"peakExecutionMemory" : 0,
"inputMetrics" : {
"bytesRead" : 0,
"recordsRead" : 0
},
"outputMetrics" : {
"bytesWritten" : 0,
"recordsWritten" : 0
},
"shuffleReadMetrics" : {
"remoteBlocksFetched" : 0,
"localBlocksFetched" : 0,
"fetchWaitTime" : 0,
"remoteBytesRead" : 0,
"remoteBytesReadToDisk" : 0,
"localBytesRead" : 0,
"recordsRead" : 0
},
"shuffleWriteMetrics" : {
"bytesWritten" : 0,
"writeTime" : 0,
"recordsWritten" : 0
}
},
"executorLogs" : {
"stdout" : "http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stdout",
"stderr" : "http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stderr"
},
"schedulerDelay" : 181,
"gettingResultTime" : 0
}, {
"taskId" : 3,
"index" : 3,
"attempt" : 0,
"launchTime" : "2016-11-15T23:20:44.053GMT",
"duration" : 725,
"executorId" : "2",
"host" : "172.22.0.111",
"status" : "FAILED",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"errorMessage" : "java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n",
"taskMetrics" : {
"executorDeserializeTime" : 0,
"executorDeserializeCpuTime" : 0,
"executorRunTime" : 456,
"executorCpuTime" : 0,
"resultSize" : 0,
"jvmGcTime" : 32,
"resultSerializationTime" : 0,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"peakExecutionMemory" : 0,
"inputMetrics" : {
"bytesRead" : 0,
"recordsRead" : 0
},
"outputMetrics" : {
"bytesWritten" : 0,
"recordsWritten" : 0
},
"shuffleReadMetrics" : {
"remoteBlocksFetched" : 0,
"localBlocksFetched" : 0,
"fetchWaitTime" : 0,
"remoteBytesRead" : 0,
"remoteBytesReadToDisk" : 0,
"localBytesRead" : 0,
"recordsRead" : 0
},
"shuffleWriteMetrics" : {
"bytesWritten" : 0,
"writeTime" : 0,
"recordsWritten" : 0
}
},
"executorLogs" : {
"stdout" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stdout",
"stderr" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stderr"
},
"schedulerDelay" : 269,
"gettingResultTime" : 0
}, {
"taskId" : 5,
"index" : 5,
"attempt" : 0,
"launchTime" : "2016-11-15T23:20:44.055GMT",
"duration" : 665,
"executorId" : "0",
"host" : "172.22.0.111",
"status" : "FAILED",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"errorMessage" : "java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n",
"taskMetrics" : {
"executorDeserializeTime" : 0,
"executorDeserializeCpuTime" : 0,
"executorRunTime" : 495,
"executorCpuTime" : 0,
"resultSize" : 0,
"jvmGcTime" : 30,
"resultSerializationTime" : 0,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"peakExecutionMemory" : 0,
"inputMetrics" : {
"bytesRead" : 0,
"recordsRead" : 0
},
"outputMetrics" : {
"bytesWritten" : 0,
"recordsWritten" : 0
},
"shuffleReadMetrics" : {
"remoteBlocksFetched" : 0,
"localBlocksFetched" : 0,
"fetchWaitTime" : 0,
"remoteBytesRead" : 0,
"remoteBytesReadToDisk" : 0,
"localBytesRead" : 0,
"recordsRead" : 0
},
"shuffleWriteMetrics" : {
"bytesWritten" : 0,
"writeTime" : 0,
"recordsWritten" : 0
}
},
"executorLogs" : {
"stdout" : "http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stdout",
"stderr" : "http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stderr"
},
"schedulerDelay" : 170,
"gettingResultTime" : 0
}, {
"taskId" : 7,
"index" : 7,
"attempt" : 0,
"launchTime" : "2016-11-15T23:20:44.056GMT",
"duration" : 685,
"executorId" : "2",
"host" : "172.22.0.111",
"status" : "FAILED",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"errorMessage" : "java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n",
"taskMetrics" : {
"executorDeserializeTime" : 0,
"executorDeserializeCpuTime" : 0,
"executorRunTime" : 448,
"executorCpuTime" : 0,
"resultSize" : 0,
"jvmGcTime" : 32,
"resultSerializationTime" : 0,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"peakExecutionMemory" : 0,
"inputMetrics" : {
"bytesRead" : 0,
"recordsRead" : 0
},
"outputMetrics" : {
"bytesWritten" : 0,
"recordsWritten" : 0
},
"shuffleReadMetrics" : {
"remoteBlocksFetched" : 0,
"localBlocksFetched" : 0,
"fetchWaitTime" : 0,
"remoteBytesRead" : 0,
"remoteBytesReadToDisk" : 0,
"localBytesRead" : 0,
"recordsRead" : 0
},
"shuffleWriteMetrics" : {
"bytesWritten" : 0,
"writeTime" : 0,
"recordsWritten" : 0
}
},
"executorLogs" : {
"stdout" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stdout",
"stderr" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stderr"
},
"schedulerDelay" : 237,
"gettingResultTime" : 0
}, {
"taskId" : 9,
"index" : 9,
"attempt" : 0,
"launchTime" : "2016-11-15T23:20:44.057GMT",
"duration" : 732,
"executorId" : "0",
"host" : "172.22.0.111",
"status" : "FAILED",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"errorMessage" : "java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n",
"taskMetrics" : {
"executorDeserializeTime" : 0,
"executorDeserializeCpuTime" : 0,
"executorRunTime" : 503,
"executorCpuTime" : 0,
"resultSize" : 0,
"jvmGcTime" : 30,
"resultSerializationTime" : 0,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"peakExecutionMemory" : 0,
"inputMetrics" : {
"bytesRead" : 0,
"recordsRead" : 0
},
"outputMetrics" : {
"bytesWritten" : 0,
"recordsWritten" : 0
},
"shuffleReadMetrics" : {
"remoteBlocksFetched" : 0,
"localBlocksFetched" : 0,
"fetchWaitTime" : 0,
"remoteBytesRead" : 0,
"remoteBytesReadToDisk" : 0,
"localBytesRead" : 0,
"recordsRead" : 0
},
"shuffleWriteMetrics" : {
"bytesWritten" : 0,
"writeTime" : 0,
"recordsWritten" : 0
}
},
"executorLogs" : {
"stdout" : "http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stdout",
"stderr" : "http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stderr"
},
"schedulerDelay" : 229,
"gettingResultTime" : 0
}, {
"taskId" : 11,
"index" : 11,
"attempt" : 0,
"launchTime" : "2016-11-15T23:20:44.058GMT",
"duration" : 678,
"executorId" : "2",
"host" : "172.22.0.111",
"status" : "FAILED",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"errorMessage" : "java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n",
"taskMetrics" : {
"executorDeserializeTime" : 0,
"executorDeserializeCpuTime" : 0,
"executorRunTime" : 451,
"executorCpuTime" : 0,
"resultSize" : 0,
"jvmGcTime" : 32,
"resultSerializationTime" : 0,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"peakExecutionMemory" : 0,
"inputMetrics" : {
"bytesRead" : 0,
"recordsRead" : 0
},
"outputMetrics" : {
"bytesWritten" : 0,
"recordsWritten" : 0
},
"shuffleReadMetrics" : {
"remoteBlocksFetched" : 0,
"localBlocksFetched" : 0,
"fetchWaitTime" : 0,
"remoteBytesRead" : 0,
"remoteBytesReadToDisk" : 0,
"localBytesRead" : 0,
"recordsRead" : 0
},
"shuffleWriteMetrics" : {
"bytesWritten" : 0,
"writeTime" : 0,
"recordsWritten" : 0
}
},
"executorLogs" : {
"stdout" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stdout",
"stderr" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stderr"
},
"schedulerDelay" : 227,
"gettingResultTime" : 0
}, {
"taskId" : 13,
"index" : 13,
"attempt" : 0,
"launchTime" : "2016-11-15T23:20:44.060GMT",
"duration" : 669,
"executorId" : "0",
"host" : "172.22.0.111",
"status" : "FAILED",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"errorMessage" : "java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n",
"taskMetrics" : {
"executorDeserializeTime" : 0,
"executorDeserializeCpuTime" : 0,
"executorRunTime" : 494,
"executorCpuTime" : 0,
"resultSize" : 0,
"jvmGcTime" : 30,
"resultSerializationTime" : 0,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"peakExecutionMemory" : 0,
"inputMetrics" : {
"bytesRead" : 0,
"recordsRead" : 0
},
"outputMetrics" : {
"bytesWritten" : 0,
"recordsWritten" : 0
},
"shuffleReadMetrics" : {
"remoteBlocksFetched" : 0,
"localBlocksFetched" : 0,
"fetchWaitTime" : 0,
"remoteBytesRead" : 0,
"remoteBytesReadToDisk" : 0,
"localBytesRead" : 0,
"recordsRead" : 0
},
"shuffleWriteMetrics" : {
"bytesWritten" : 0,
"writeTime" : 0,
"recordsWritten" : 0
}
},
"executorLogs" : {
"stdout" : "http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stdout",
"stderr" : "http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stderr"
},
"schedulerDelay" : 175,
"gettingResultTime" : 0
}, {
"taskId" : 15,
"index" : 15,
"attempt" : 0,
"launchTime" : "2016-11-15T23:20:44.065GMT",
"duration" : 672,
"executorId" : "2",
"host" : "172.22.0.111",
"status" : "FAILED",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"errorMessage" : "java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n",
"taskMetrics" : {
"executorDeserializeTime" : 0,
"executorDeserializeCpuTime" : 0,
"executorRunTime" : 446,
"executorCpuTime" : 0,
"resultSize" : 0,
"jvmGcTime" : 32,
"resultSerializationTime" : 0,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"peakExecutionMemory" : 0,
"inputMetrics" : {
"bytesRead" : 0,
"recordsRead" : 0
},
"outputMetrics" : {
"bytesWritten" : 0,
"recordsWritten" : 0
},
"shuffleReadMetrics" : {
"remoteBlocksFetched" : 0,
"localBlocksFetched" : 0,
"fetchWaitTime" : 0,
"remoteBytesRead" : 0,
"remoteBytesReadToDisk" : 0,
"localBytesRead" : 0,
"recordsRead" : 0
},
"shuffleWriteMetrics" : {
"bytesWritten" : 0,
"writeTime" : 0,
"recordsWritten" : 0
}
},
"executorLogs" : {
"stdout" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stdout",
"stderr" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stderr"
},
"schedulerDelay" : 226,
"gettingResultTime" : 0
}, {
"taskId" : 19,
"index" : 11,
"attempt" : 1,
"launchTime" : "2016-11-15T23:20:44.736GMT",
"duration" : 13,
"executorId" : "2",
"host" : "172.22.0.111",
"status" : "FAILED",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"errorMessage" : "java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n",
"taskMetrics" : {
"executorDeserializeTime" : 0,
"executorDeserializeCpuTime" : 0,
"executorRunTime" : 2,
"executorCpuTime" : 0,
"resultSize" : 0,
"jvmGcTime" : 0,
"resultSerializationTime" : 0,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"peakExecutionMemory" : 0,
"inputMetrics" : {
"bytesRead" : 0,
"recordsRead" : 0
},
"outputMetrics" : {
"bytesWritten" : 0,
"recordsWritten" : 0
},
"shuffleReadMetrics" : {
"remoteBlocksFetched" : 0,
"localBlocksFetched" : 0,
"fetchWaitTime" : 0,
"remoteBytesRead" : 0,
"remoteBytesReadToDisk" : 0,
"localBytesRead" : 0,
"recordsRead" : 0
},
"shuffleWriteMetrics" : {
"bytesWritten" : 0,
"writeTime" : 0,
"recordsWritten" : 0
}
},
"executorLogs" : {
"stdout" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stdout",
"stderr" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stderr"
},
"schedulerDelay" : 11,
"gettingResultTime" : 0
}, {
"taskId" : 20,
"index" : 15,
"attempt" : 1,
"launchTime" : "2016-11-15T23:20:44.737GMT",
"duration" : 19,
"executorId" : "2",
"host" : "172.22.0.111",
"status" : "FAILED",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"errorMessage" : "java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n",
"taskMetrics" : {
"executorDeserializeTime" : 0,
"executorDeserializeCpuTime" : 0,
"executorRunTime" : 10,
"executorCpuTime" : 0,
"resultSize" : 0,
"jvmGcTime" : 0,
"resultSerializationTime" : 0,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"peakExecutionMemory" : 0,
"inputMetrics" : {
"bytesRead" : 0,
"recordsRead" : 0
},
"outputMetrics" : {
"bytesWritten" : 0,
"recordsWritten" : 0
},
"shuffleReadMetrics" : {
"remoteBlocksFetched" : 0,
"localBlocksFetched" : 0,
"fetchWaitTime" : 0,
"remoteBytesRead" : 0,
"remoteBytesReadToDisk" : 0,
"localBytesRead" : 0,
"recordsRead" : 0
},
"shuffleWriteMetrics" : {
"bytesWritten" : 0,
"writeTime" : 0,
"recordsWritten" : 0
}
},
"executorLogs" : {
"stdout" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stdout",
"stderr" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stderr"
},
"schedulerDelay" : 9,
"gettingResultTime" : 0
} ]

View file

@ -154,6 +154,12 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers
"applications/local-1430917381534/stages/0/0/taskList?sortBy=-runtime",
"stage task list w/ sortBy short names: runtime" ->
"applications/local-1430917381534/stages/0/0/taskList?sortBy=runtime",
"stage task list w/ status" ->
"applications/app-20161115172038-0000/stages/0/0/taskList?status=failed",
"stage task list w/ status & offset & length" ->
"applications/local-1430917381534/stages/0/0/taskList?status=success&offset=1&length=2",
"stage task list w/ status & sortBy short names: runtime" ->
"applications/local-1430917381534/stages/0/0/taskList?status=success&sortBy=runtime",
"stage list with accumulable json" -> "applications/local-1426533911241/1/stages",
"stage with accumulable json" -> "applications/local-1426533911241/1/stages/0/0",

View file

@ -499,7 +499,8 @@ can be identified by their `[attempt-id]`. In the API listed below, when running
A list of all tasks for the given stage attempt.
<br><code>?offset=[offset]&amp;length=[len]</code> list tasks in the given range.
<br><code>?sortBy=[runtime|-runtime]</code> sort the tasks.
<br>Example: <code>?offset=10&amp;length=50&amp;sortBy=runtime</code>
<br><code>?status=[running|success|killed|failed|unknown]</code> list only tasks in the state.
<br>Example: <code>?offset=10&amp;length=50&amp;sortBy=runtime&amp;status=running</code>
</td>
</tr>
<tr>