[SPARK-22577][CORE] executor page blacklist status should update with TaskSet level blacklisting

## What changes were proposed in this pull request?

In this PR stage blacklisting is propagated to UI by introducing a new Spark listener event (SparkListenerExecutorBlacklistedForStage) which indicates the executor is blacklisted for a stage. Either because of the number of failures are exceeded a limit given for an executor (spark.blacklist.stage.maxFailedTasksPerExecutor) or because of the whole node is blacklisted for a stage (spark.blacklist.stage.maxFailedExecutorsPerNode). In case of the node is blacklisting all executors will listed as blacklisted for the stage.

Blacklisting state for a selected stage can be seen "Aggregated Metrics by Executor" table's blacklisting column, where after this change three possible labels could be found:
- "for application": when the executor is blacklisted for the application (see the configuration spark.blacklist.application.maxFailedTasksPerExecutor for details)
- "for stage": when the executor is **only** blacklisted for the stage
- "false" : when the executor is not blacklisted at all

## How was this patch tested?

It is tested both manually and with unit tests.

#### Unit tests

- HistoryServerSuite
- TaskSetBlacklistSuite
- AppStatusListenerSuite

#### Manual test for executor blacklisting

Running Spark as a local cluster:
```
$ bin/spark-shell --master "local-cluster[2,1,1024]" --conf "spark.blacklist.enabled=true" --conf "spark.blacklist.stage.maxFailedTasksPerExecutor=1" --conf "spark.blacklist.application.maxFailedTasksPerExecutor=10" --conf "spark.eventLog.enabled=true"
```

Executing:
``` scala
import org.apache.spark.SparkEnv

sc.parallelize(1 to 10, 10).map { x =>
  if (SparkEnv.get.executorId == "0") throw new RuntimeException("Bad executor")
  else (x % 3, x)
}.reduceByKey((a, b) => a + b).collect()
```

To see result check the "Aggregated Metrics by Executor" section at the bottom of picture:

![UI screenshot for stage level blacklisting executor](https://issues.apache.org/jira/secure/attachment/12905283/stage_blacklisting.png)

#### Manual test for node blacklisting

Running Spark as on a cluster:

``` bash
./bin/spark-shell --master yarn --deploy-mode client --executor-memory=2G --num-executors=8 --conf "spark.blacklist.enabled=true" --conf "spark.blacklist.stage.maxFailedTasksPerExecutor=1" --conf "spark.blacklist.stage.maxFailedExecutorsPerNode=1"  --conf "spark.blacklist.application.maxFailedTasksPerExecutor=10" --conf "spark.eventLog.enabled=true"
```

And the job was:

``` scala
import org.apache.spark.SparkEnv

sc.parallelize(1 to 10000, 10).map { x =>
  if (SparkEnv.get.executorId.toInt >= 4) throw new RuntimeException("Bad executor")
    else (x % 3, x)
}.reduceByKey((a, b) => a + b).collect()
```

The result is:

![UI screenshot for stage level node blacklisting](https://issues.apache.org/jira/secure/attachment/12906833/node_blacklisting_for_stage.png)

Here you can see apiros3.gce.test.com was node blacklisted for the stage because of failures on executor 4 and 5. As expected executor 3 is also blacklisted even it has no failures itself but sharing the node with 4 and 5.

Author: “attilapiros” <piros.attila.zsolt@gmail.com>
Author: Attila Zsolt Piros <2017933+attilapiros@users.noreply.github.com>

Closes #20203 from attilapiros/SPARK-22577.
This commit is contained in:
“attilapiros” 2018-01-24 11:34:59 -06:00 committed by Imran Rashid
parent de36f65d3a
commit 0ec95bb7df
27 changed files with 2047 additions and 110 deletions

View file

@ -118,6 +118,18 @@ public class SparkFirehoseListener implements SparkListenerInterface {
onEvent(executorBlacklisted);
}
@Override
public void onExecutorBlacklistedForStage(
SparkListenerExecutorBlacklistedForStage executorBlacklistedForStage) {
onEvent(executorBlacklistedForStage);
}
@Override
public void onNodeBlacklistedForStage(
SparkListenerNodeBlacklistedForStage nodeBlacklistedForStage) {
onEvent(nodeBlacklistedForStage);
}
@Override
public final void onExecutorUnblacklisted(
SparkListenerExecutorUnblacklisted executorUnblacklisted) {

View file

@ -207,6 +207,15 @@ private[spark] class EventLoggingListener(
logEvent(event, flushLogger = true)
}
override def onExecutorBlacklistedForStage(
event: SparkListenerExecutorBlacklistedForStage): Unit = {
logEvent(event, flushLogger = true)
}
override def onNodeBlacklistedForStage(event: SparkListenerNodeBlacklistedForStage): Unit = {
logEvent(event, flushLogger = true)
}
override def onExecutorUnblacklisted(event: SparkListenerExecutorUnblacklisted): Unit = {
logEvent(event, flushLogger = true)
}

View file

@ -120,6 +120,24 @@ case class SparkListenerExecutorBlacklisted(
taskFailures: Int)
extends SparkListenerEvent
@DeveloperApi
case class SparkListenerExecutorBlacklistedForStage(
time: Long,
executorId: String,
taskFailures: Int,
stageId: Int,
stageAttemptId: Int)
extends SparkListenerEvent
@DeveloperApi
case class SparkListenerNodeBlacklistedForStage(
time: Long,
hostId: String,
executorFailures: Int,
stageId: Int,
stageAttemptId: Int)
extends SparkListenerEvent
@DeveloperApi
case class SparkListenerExecutorUnblacklisted(time: Long, executorId: String)
extends SparkListenerEvent
@ -261,6 +279,17 @@ private[spark] trait SparkListenerInterface {
*/
def onExecutorBlacklisted(executorBlacklisted: SparkListenerExecutorBlacklisted): Unit
/**
* Called when the driver blacklists an executor for a stage.
*/
def onExecutorBlacklistedForStage(
executorBlacklistedForStage: SparkListenerExecutorBlacklistedForStage): Unit
/**
* Called when the driver blacklists a node for a stage.
*/
def onNodeBlacklistedForStage(nodeBlacklistedForStage: SparkListenerNodeBlacklistedForStage): Unit
/**
* Called when the driver re-enables a previously blacklisted executor.
*/
@ -339,6 +368,12 @@ abstract class SparkListener extends SparkListenerInterface {
override def onExecutorBlacklisted(
executorBlacklisted: SparkListenerExecutorBlacklisted): Unit = { }
def onExecutorBlacklistedForStage(
executorBlacklistedForStage: SparkListenerExecutorBlacklistedForStage): Unit = { }
def onNodeBlacklistedForStage(
nodeBlacklistedForStage: SparkListenerNodeBlacklistedForStage): Unit = { }
override def onExecutorUnblacklisted(
executorUnblacklisted: SparkListenerExecutorUnblacklisted): Unit = { }

View file

@ -61,6 +61,10 @@ private[spark] trait SparkListenerBus
listener.onExecutorAdded(executorAdded)
case executorRemoved: SparkListenerExecutorRemoved =>
listener.onExecutorRemoved(executorRemoved)
case executorBlacklistedForStage: SparkListenerExecutorBlacklistedForStage =>
listener.onExecutorBlacklistedForStage(executorBlacklistedForStage)
case nodeBlacklistedForStage: SparkListenerNodeBlacklistedForStage =>
listener.onNodeBlacklistedForStage(nodeBlacklistedForStage)
case executorBlacklisted: SparkListenerExecutorBlacklisted =>
listener.onExecutorBlacklisted(executorBlacklisted)
case executorUnblacklisted: SparkListenerExecutorUnblacklisted =>

View file

@ -36,8 +36,12 @@ import org.apache.spark.util.Clock
* [[TaskSetManager]] this class is designed only to be called from code with a lock on the
* TaskScheduler (e.g. its event handlers). It should not be called from other threads.
*/
private[scheduler] class TaskSetBlacklist(val conf: SparkConf, val stageId: Int, val clock: Clock)
extends Logging {
private[scheduler] class TaskSetBlacklist(
private val listenerBus: LiveListenerBus,
val conf: SparkConf,
val stageId: Int,
val stageAttemptId: Int,
val clock: Clock) extends Logging {
private val MAX_TASK_ATTEMPTS_PER_EXECUTOR = conf.get(config.MAX_TASK_ATTEMPTS_PER_EXECUTOR)
private val MAX_TASK_ATTEMPTS_PER_NODE = conf.get(config.MAX_TASK_ATTEMPTS_PER_NODE)
@ -128,16 +132,23 @@ private[scheduler] class TaskSetBlacklist(val conf: SparkConf, val stageId: Int,
}
// Check if enough tasks have failed on the executor to blacklist it for the entire stage.
if (execFailures.numUniqueTasksWithFailures >= MAX_FAILURES_PER_EXEC_STAGE) {
val numFailures = execFailures.numUniqueTasksWithFailures
if (numFailures >= MAX_FAILURES_PER_EXEC_STAGE) {
if (blacklistedExecs.add(exec)) {
logInfo(s"Blacklisting executor ${exec} for stage $stageId")
// This executor has been pushed into the blacklist for this stage. Let's check if it
// pushes the whole node into the blacklist.
val blacklistedExecutorsOnNode =
execsWithFailuresOnNode.filter(blacklistedExecs.contains(_))
if (blacklistedExecutorsOnNode.size >= MAX_FAILED_EXEC_PER_NODE_STAGE) {
val now = clock.getTimeMillis()
listenerBus.post(
SparkListenerExecutorBlacklistedForStage(now, exec, numFailures, stageId, stageAttemptId))
val numFailExec = blacklistedExecutorsOnNode.size
if (numFailExec >= MAX_FAILED_EXEC_PER_NODE_STAGE) {
if (blacklistedNodes.add(host)) {
logInfo(s"Blacklisting ${host} for stage $stageId")
listenerBus.post(
SparkListenerNodeBlacklistedForStage(now, host, numFailExec, stageId, stageAttemptId))
}
}
}

View file

@ -102,7 +102,7 @@ private[spark] class TaskSetManager(
private[scheduler] val taskSetBlacklistHelperOpt: Option[TaskSetBlacklist] = {
blacklistTracker.map { _ =>
new TaskSetBlacklist(conf, stageId, clock)
new TaskSetBlacklist(sched.sc.listenerBus, conf, stageId, taskSet.stageAttemptId, clock)
}
}

View file

@ -211,6 +211,31 @@ private[spark] class AppStatusListener(
updateBlackListStatus(event.executorId, true)
}
override def onExecutorBlacklistedForStage(
event: SparkListenerExecutorBlacklistedForStage): Unit = {
Option(liveStages.get((event.stageId, event.stageAttemptId))).foreach { stage =>
val now = System.nanoTime()
val esummary = stage.executorSummary(event.executorId)
esummary.isBlacklisted = true
maybeUpdate(esummary, now)
}
}
override def onNodeBlacklistedForStage(event: SparkListenerNodeBlacklistedForStage): Unit = {
val now = System.nanoTime()
// Implicitly blacklist every available executor for the stage associated with this node
Option(liveStages.get((event.stageId, event.stageAttemptId))).foreach { stage =>
liveExecutors.values.foreach { exec =>
if (exec.hostname == event.hostId) {
val esummary = stage.executorSummary(exec.executorId)
esummary.isBlacklisted = true
maybeUpdate(esummary, now)
}
}
}
}
override def onExecutorUnblacklisted(event: SparkListenerExecutorUnblacklisted): Unit = {
updateBlackListStatus(event.executorId, false)
}

View file

@ -316,6 +316,7 @@ private class LiveExecutorStageSummary(
var succeededTasks = 0
var failedTasks = 0
var killedTasks = 0
var isBlacklisted = false
var metrics = createMetrics(default = 0L)
@ -334,7 +335,8 @@ private class LiveExecutorStageSummary(
metrics.shuffleWriteMetrics.bytesWritten,
metrics.shuffleWriteMetrics.recordsWritten,
metrics.memoryBytesSpilled,
metrics.diskBytesSpilled)
metrics.diskBytesSpilled,
isBlacklisted)
new ExecutorStageSummaryWrapper(stageId, attemptId, executorId, info)
}

View file

@ -68,7 +68,8 @@ class ExecutorStageSummary private[spark](
val shuffleWrite : Long,
val shuffleWriteRecords : Long,
val memoryBytesSpilled : Long,
val diskBytesSpilled : Long)
val diskBytesSpilled : Long,
val isBlacklistedForStage: Boolean)
class ExecutorSummary private[spark](
val id: String,

View file

@ -136,7 +136,15 @@ private[ui] class ExecutorTable(stage: StageData, store: AppStatusStore) {
{Utils.bytesToString(v.diskBytesSpilled)}
</td>
}}
<td>{executor.map(_.isBlacklisted).getOrElse(false)}</td>
{
if (executor.map(_.isBlacklisted).getOrElse(false)) {
<td>for application</td>
} else if (v.isBlacklistedForStage) {
<td>for stage</td>
} else {
<td>false</td>
}
}
</tr>
}
}

View file

@ -1,4 +1,34 @@
[ {
"id" : "application_1516285256255_0012",
"name" : "Spark shell",
"attempts" : [ {
"startTime" : "2018-01-18T18:30:35.119GMT",
"endTime" : "2018-01-18T18:38:27.938GMT",
"lastUpdated" : "",
"duration" : 472819,
"sparkUser" : "attilapiros",
"completed" : true,
"appSparkVersion" : "2.3.0-SNAPSHOT",
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1516300235119,
"endTimeEpoch" : 1516300707938
} ]
}, {
"id" : "app-20180109111548-0000",
"name" : "Spark shell",
"attempts" : [ {
"startTime" : "2018-01-09T10:15:42.372GMT",
"endTime" : "2018-01-09T10:24:37.606GMT",
"lastUpdated" : "",
"duration" : 535234,
"sparkUser" : "attilapiros",
"completed" : true,
"appSparkVersion" : "2.3.0-SNAPSHOT",
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1515492942372,
"endTimeEpoch" : 1515493477606
} ]
}, {
"id" : "app-20161116163331-0000",
"name" : "Spark shell",
"attempts" : [ {
@ -9,9 +39,9 @@
"sparkUser" : "jose",
"completed" : true,
"appSparkVersion" : "2.1.0-SNAPSHOT",
"endTimeEpoch" : 1479335620587,
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1479335609916,
"lastUpdatedEpoch" : 0
"endTimeEpoch" : 1479335620587
} ]
}, {
"id" : "app-20161115172038-0000",
@ -24,9 +54,9 @@
"sparkUser" : "jose",
"completed" : true,
"appSparkVersion" : "2.1.0-SNAPSHOT",
"endTimeEpoch" : 1479252138874,
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1479252037079,
"lastUpdatedEpoch" : 0
"endTimeEpoch" : 1479252138874
} ]
}, {
"id" : "local-1430917381534",
@ -39,9 +69,9 @@
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "1.4.0-SNAPSHOT",
"endTimeEpoch" : 1430917391398,
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1430917380893,
"lastUpdatedEpoch" : 0
"endTimeEpoch" : 1430917391398
} ]
}, {
"id" : "local-1430917381535",
@ -55,9 +85,9 @@
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "1.4.0-SNAPSHOT",
"endTimeEpoch" : 1430917380950,
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1430917380893,
"lastUpdatedEpoch" : 0
"endTimeEpoch" : 1430917380950
}, {
"attemptId" : "1",
"startTime" : "2015-05-06T13:03:00.880GMT",
@ -67,9 +97,9 @@
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "1.4.0-SNAPSHOT",
"endTimeEpoch" : 1430917380890,
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1430917380880,
"lastUpdatedEpoch" : 0
"endTimeEpoch" : 1430917380890
} ]
}, {
"id" : "local-1426533911241",
@ -83,9 +113,9 @@
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "",
"endTimeEpoch" : 1426633945177,
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1426633910242,
"lastUpdatedEpoch" : 0
"endTimeEpoch" : 1426633945177
}, {
"attemptId" : "1",
"startTime" : "2015-03-16T19:25:10.242GMT",
@ -95,9 +125,9 @@
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "",
"endTimeEpoch" : 1426533945177,
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1426533910242,
"lastUpdatedEpoch" : 0
"endTimeEpoch" : 1426533945177
} ]
}, {
"id" : "local-1425081759269",
@ -110,9 +140,9 @@
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "",
"endTimeEpoch" : 1425081766912,
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1425081758277,
"lastUpdatedEpoch" : 0
"endTimeEpoch" : 1425081766912
} ]
}, {
"id" : "local-1422981780767",
@ -125,9 +155,9 @@
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "",
"endTimeEpoch" : 1422981788731,
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1422981779720,
"lastUpdatedEpoch" : 0
"endTimeEpoch" : 1422981788731
} ]
}, {
"id" : "local-1422981759269",
@ -140,8 +170,8 @@
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "",
"endTimeEpoch" : 1422981766912,
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1422981758277,
"lastUpdatedEpoch" : 0
"endTimeEpoch" : 1422981766912
} ]
} ]

View file

@ -0,0 +1,639 @@
{
"status": "COMPLETE",
"stageId": 0,
"attemptId": 0,
"numTasks": 10,
"numActiveTasks": 0,
"numCompleteTasks": 10,
"numFailedTasks": 2,
"numKilledTasks": 0,
"numCompletedIndices": 10,
"executorRunTime": 761,
"executorCpuTime": 269916000,
"submissionTime": "2018-01-09T10:21:18.152GMT",
"firstTaskLaunchedTime": "2018-01-09T10:21:18.347GMT",
"completionTime": "2018-01-09T10:21:19.062GMT",
"inputBytes": 0,
"inputRecords": 0,
"outputBytes": 0,
"outputRecords": 0,
"shuffleReadBytes": 0,
"shuffleReadRecords": 0,
"shuffleWriteBytes": 460,
"shuffleWriteRecords": 10,
"memoryBytesSpilled": 0,
"diskBytesSpilled": 0,
"name": "map at <console>:26",
"details": "org.apache.spark.rdd.RDD.map(RDD.scala:370)\n$line17.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:26)\n$line17.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:34)\n$line17.$read$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:36)\n$line17.$read$$iw$$iw$$iw$$iw$$iw.<init>(<console>:38)\n$line17.$read$$iw$$iw$$iw$$iw.<init>(<console>:40)\n$line17.$read$$iw$$iw$$iw.<init>(<console>:42)\n$line17.$read$$iw$$iw.<init>(<console>:44)\n$line17.$read$$iw.<init>(<console>:46)\n$line17.$read.<init>(<console>:48)\n$line17.$read$.<init>(<console>:52)\n$line17.$read$.<clinit>(<console>)\n$line17.$eval$.$print$lzycompute(<console>:7)\n$line17.$eval$.$print(<console>:6)\n$line17.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:786)",
"schedulingPool": "default",
"rddIds": [
1,
0
],
"accumulatorUpdates": [],
"tasks": {
"0": {
"taskId": 0,
"index": 0,
"attempt": 0,
"launchTime": "2018-01-09T10:21:18.347GMT",
"duration": 562,
"executorId": "0",
"host": "172.30.65.138",
"status": "FAILED",
"taskLocality": "PROCESS_LOCAL",
"speculative": false,
"accumulatorUpdates": [],
"errorMessage": "java.lang.RuntimeException: Bad executor\n\tat $line17.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$2.apply(<console>:27)\n\tat $line17.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$2.apply(<console>:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.collection.ExternalSorter.insertAll(ExternalSorter.scala:193)\n\tat org.apache.spark.shuffle.sort.SortShuffleWriter.write(SortShuffleWriter.scala:63)\n\tat org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:96)\n\tat org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:109)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:345)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\n\tat java.lang.Thread.run(Thread.java:748)\n",
"taskMetrics": {
"executorDeserializeTime": 0,
"executorDeserializeCpuTime": 0,
"executorRunTime": 460,
"executorCpuTime": 0,
"resultSize": 0,
"jvmGcTime": 14,
"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": 3873006,
"recordsWritten": 0
}
}
},
"5": {
"taskId": 5,
"index": 3,
"attempt": 0,
"launchTime": "2018-01-09T10:21:18.958GMT",
"duration": 22,
"executorId": "1",
"host": "172.30.65.138",
"status": "SUCCESS",
"taskLocality": "PROCESS_LOCAL",
"speculative": false,
"accumulatorUpdates": [],
"taskMetrics": {
"executorDeserializeTime": 3,
"executorDeserializeCpuTime": 2586000,
"executorRunTime": 9,
"executorCpuTime": 9635000,
"resultSize": 1029,
"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": 46,
"writeTime": 262919,
"recordsWritten": 1
}
}
},
"10": {
"taskId": 10,
"index": 8,
"attempt": 0,
"launchTime": "2018-01-09T10:21:19.034GMT",
"duration": 12,
"executorId": "1",
"host": "172.30.65.138",
"status": "SUCCESS",
"taskLocality": "PROCESS_LOCAL",
"speculative": false,
"accumulatorUpdates": [],
"taskMetrics": {
"executorDeserializeTime": 2,
"executorDeserializeCpuTime": 1803000,
"executorRunTime": 6,
"executorCpuTime": 6157000,
"resultSize": 1029,
"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": 46,
"writeTime": 243647,
"recordsWritten": 1
}
}
},
"1": {
"taskId": 1,
"index": 1,
"attempt": 0,
"launchTime": "2018-01-09T10:21:18.364GMT",
"duration": 565,
"executorId": "1",
"host": "172.30.65.138",
"status": "SUCCESS",
"taskLocality": "PROCESS_LOCAL",
"speculative": false,
"accumulatorUpdates": [],
"taskMetrics": {
"executorDeserializeTime": 301,
"executorDeserializeCpuTime": 200029000,
"executorRunTime": 212,
"executorCpuTime": 198479000,
"resultSize": 1115,
"jvmGcTime": 13,
"resultSerializationTime": 1,
"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": 46,
"writeTime": 2409488,
"recordsWritten": 1
}
}
},
"6": {
"taskId": 6,
"index": 4,
"attempt": 0,
"launchTime": "2018-01-09T10:21:18.980GMT",
"duration": 16,
"executorId": "1",
"host": "172.30.65.138",
"status": "SUCCESS",
"taskLocality": "PROCESS_LOCAL",
"speculative": false,
"accumulatorUpdates": [],
"taskMetrics": {
"executorDeserializeTime": 3,
"executorDeserializeCpuTime": 2610000,
"executorRunTime": 10,
"executorCpuTime": 9622000,
"resultSize": 1029,
"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": 46,
"writeTime": 385110,
"recordsWritten": 1
}
}
},
"9": {
"taskId": 9,
"index": 7,
"attempt": 0,
"launchTime": "2018-01-09T10:21:19.022GMT",
"duration": 12,
"executorId": "1",
"host": "172.30.65.138",
"status": "SUCCESS",
"taskLocality": "PROCESS_LOCAL",
"speculative": false,
"accumulatorUpdates": [],
"taskMetrics": {
"executorDeserializeTime": 2,
"executorDeserializeCpuTime": 1981000,
"executorRunTime": 7,
"executorCpuTime": 6335000,
"resultSize": 1029,
"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": 46,
"writeTime": 259354,
"recordsWritten": 1
}
}
},
"2": {
"taskId": 2,
"index": 2,
"attempt": 0,
"launchTime": "2018-01-09T10:21:18.899GMT",
"duration": 27,
"executorId": "0",
"host": "172.30.65.138",
"status": "FAILED",
"taskLocality": "PROCESS_LOCAL",
"speculative": false,
"accumulatorUpdates": [],
"errorMessage": "java.lang.RuntimeException: Bad executor\n\tat $line17.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$2.apply(<console>:27)\n\tat $line17.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$2.apply(<console>:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.collection.ExternalSorter.insertAll(ExternalSorter.scala:193)\n\tat org.apache.spark.shuffle.sort.SortShuffleWriter.write(SortShuffleWriter.scala:63)\n\tat org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:96)\n\tat org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:109)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:345)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\n\tat java.lang.Thread.run(Thread.java:748)\n",
"taskMetrics": {
"executorDeserializeTime": 0,
"executorDeserializeCpuTime": 0,
"executorRunTime": 16,
"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": 126128,
"recordsWritten": 0
}
}
},
"7": {
"taskId": 7,
"index": 5,
"attempt": 0,
"launchTime": "2018-01-09T10:21:18.996GMT",
"duration": 15,
"executorId": "1",
"host": "172.30.65.138",
"status": "SUCCESS",
"taskLocality": "PROCESS_LOCAL",
"speculative": false,
"accumulatorUpdates": [],
"taskMetrics": {
"executorDeserializeTime": 2,
"executorDeserializeCpuTime": 2231000,
"executorRunTime": 9,
"executorCpuTime": 8407000,
"resultSize": 1029,
"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": 46,
"writeTime": 205520,
"recordsWritten": 1
}
}
},
"3": {
"taskId": 3,
"index": 0,
"attempt": 1,
"launchTime": "2018-01-09T10:21:18.919GMT",
"duration": 24,
"executorId": "1",
"host": "172.30.65.138",
"status": "SUCCESS",
"taskLocality": "PROCESS_LOCAL",
"speculative": false,
"accumulatorUpdates": [],
"taskMetrics": {
"executorDeserializeTime": 8,
"executorDeserializeCpuTime": 8878000,
"executorRunTime": 10,
"executorCpuTime": 9364000,
"resultSize": 1029,
"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": 46,
"writeTime": 207014,
"recordsWritten": 1
}
}
},
"11": {
"taskId": 11,
"index": 9,
"attempt": 0,
"launchTime": "2018-01-09T10:21:19.045GMT",
"duration": 15,
"executorId": "1",
"host": "172.30.65.138",
"status": "SUCCESS",
"taskLocality": "PROCESS_LOCAL",
"speculative": false,
"accumulatorUpdates": [],
"taskMetrics": {
"executorDeserializeTime": 3,
"executorDeserializeCpuTime": 2017000,
"executorRunTime": 6,
"executorCpuTime": 6676000,
"resultSize": 1029,
"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": 46,
"writeTime": 233652,
"recordsWritten": 1
}
}
},
"8": {
"taskId": 8,
"index": 6,
"attempt": 0,
"launchTime": "2018-01-09T10:21:19.011GMT",
"duration": 11,
"executorId": "1",
"host": "172.30.65.138",
"status": "SUCCESS",
"taskLocality": "PROCESS_LOCAL",
"speculative": false,
"accumulatorUpdates": [],
"taskMetrics": {
"executorDeserializeTime": 1,
"executorDeserializeCpuTime": 1554000,
"executorRunTime": 7,
"executorCpuTime": 6034000,
"resultSize": 1029,
"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": 46,
"writeTime": 213296,
"recordsWritten": 1
}
}
},
"4": {
"taskId": 4,
"index": 2,
"attempt": 1,
"launchTime": "2018-01-09T10:21:18.943GMT",
"duration": 16,
"executorId": "1",
"host": "172.30.65.138",
"status": "SUCCESS",
"taskLocality": "PROCESS_LOCAL",
"speculative": false,
"accumulatorUpdates": [],
"taskMetrics": {
"executorDeserializeTime": 2,
"executorDeserializeCpuTime": 2211000,
"executorRunTime": 9,
"executorCpuTime": 9207000,
"resultSize": 1029,
"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": 46,
"writeTime": 292381,
"recordsWritten": 1
}
}
}
},
"executorSummary": {
"0": {
"taskTime": 589,
"failedTasks": 2,
"succeededTasks": 0,
"killedTasks": 0,
"inputBytes": 0,
"inputRecords": 0,
"outputBytes": 0,
"outputRecords": 0,
"shuffleRead": 0,
"shuffleReadRecords": 0,
"shuffleWrite": 0,
"shuffleWriteRecords": 0,
"memoryBytesSpilled": 0,
"diskBytesSpilled": 0,
"isBlacklistedForStage": true
},
"1": {
"taskTime": 708,
"failedTasks": 0,
"succeededTasks": 10,
"killedTasks": 0,
"inputBytes": 0,
"inputRecords": 0,
"outputBytes": 0,
"outputRecords": 0,
"shuffleRead": 0,
"shuffleReadRecords": 0,
"shuffleWrite": 460,
"shuffleWriteRecords": 10,
"memoryBytesSpilled": 0,
"diskBytesSpilled": 0,
"isBlacklistedForStage": false
}
},
"killedTasksSummary": {}
}

View file

@ -0,0 +1,783 @@
{
"status" : "COMPLETE",
"stageId" : 0,
"attemptId" : 0,
"numTasks" : 10,
"numActiveTasks" : 0,
"numCompleteTasks" : 10,
"numFailedTasks" : 4,
"numKilledTasks" : 0,
"numCompletedIndices" : 10,
"executorRunTime" : 5080,
"executorCpuTime" : 1163210819,
"submissionTime" : "2018-01-18T18:33:12.658GMT",
"firstTaskLaunchedTime" : "2018-01-18T18:33:12.816GMT",
"completionTime" : "2018-01-18T18:33:15.279GMT",
"inputBytes" : 0,
"inputRecords" : 0,
"outputBytes" : 0,
"outputRecords" : 0,
"shuffleReadBytes" : 0,
"shuffleReadRecords" : 0,
"shuffleWriteBytes" : 1461,
"shuffleWriteRecords" : 30,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"name" : "map at <console>:27",
"details" : "org.apache.spark.rdd.RDD.map(RDD.scala:370)\n$line15.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:27)\n$line15.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:35)\n$line15.$read$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:37)\n$line15.$read$$iw$$iw$$iw$$iw$$iw.<init>(<console>:39)\n$line15.$read$$iw$$iw$$iw$$iw.<init>(<console>:41)\n$line15.$read$$iw$$iw$$iw.<init>(<console>:43)\n$line15.$read$$iw$$iw.<init>(<console>:45)\n$line15.$read$$iw.<init>(<console>:47)\n$line15.$read.<init>(<console>:49)\n$line15.$read$.<init>(<console>:53)\n$line15.$read$.<clinit>(<console>)\n$line15.$eval$.$print$lzycompute(<console>:7)\n$line15.$eval$.$print(<console>:6)\n$line15.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:786)",
"schedulingPool" : "default",
"rddIds" : [ 1, 0 ],
"accumulatorUpdates" : [ ],
"tasks" : {
"0" : {
"taskId" : 0,
"index" : 0,
"attempt" : 0,
"launchTime" : "2018-01-18T18:33:12.816GMT",
"duration" : 2064,
"executorId" : "1",
"host" : "apiros-3.gce.test.com",
"status" : "SUCCESS",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"taskMetrics" : {
"executorDeserializeTime" : 1081,
"executorDeserializeCpuTime" : 353981050,
"executorRunTime" : 914,
"executorCpuTime" : 368865439,
"resultSize" : 1134,
"jvmGcTime" : 75,
"resultSerializationTime" : 1,
"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" : 144,
"writeTime" : 3662221,
"recordsWritten" : 3
}
}
},
"5" : {
"taskId" : 5,
"index" : 5,
"attempt" : 0,
"launchTime" : "2018-01-18T18:33:14.320GMT",
"duration" : 73,
"executorId" : "5",
"host" : "apiros-2.gce.test.com",
"status" : "FAILED",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"errorMessage" : "java.lang.RuntimeException: Bad executor\n\tat $line15.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$2.apply(<console>:28)\n\tat $line15.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$2.apply(<console>:27)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.collection.ExternalSorter.insertAll(ExternalSorter.scala:193)\n\tat org.apache.spark.shuffle.sort.SortShuffleWriter.write(SortShuffleWriter.scala:63)\n\tat org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:96)\n\tat org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:109)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:345)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\n\tat java.lang.Thread.run(Thread.java:748)\n",
"taskMetrics" : {
"executorDeserializeTime" : 0,
"executorDeserializeCpuTime" : 0,
"executorRunTime" : 27,
"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" : 191901,
"recordsWritten" : 0
}
}
},
"10" : {
"taskId" : 10,
"index" : 1,
"attempt" : 1,
"launchTime" : "2018-01-18T18:33:15.069GMT",
"duration" : 132,
"executorId" : "2",
"host" : "apiros-3.gce.test.com",
"status" : "SUCCESS",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"taskMetrics" : {
"executorDeserializeTime" : 5,
"executorDeserializeCpuTime" : 4598966,
"executorRunTime" : 76,
"executorCpuTime" : 20826337,
"resultSize" : 1091,
"jvmGcTime" : 0,
"resultSerializationTime" : 1,
"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" : 144,
"writeTime" : 301705,
"recordsWritten" : 3
}
}
},
"1" : {
"taskId" : 1,
"index" : 1,
"attempt" : 0,
"launchTime" : "2018-01-18T18:33:12.832GMT",
"duration" : 1506,
"executorId" : "5",
"host" : "apiros-2.gce.test.com",
"status" : "FAILED",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"errorMessage" : "java.lang.RuntimeException: Bad executor\n\tat $line15.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$2.apply(<console>:28)\n\tat $line15.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$2.apply(<console>:27)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.collection.ExternalSorter.insertAll(ExternalSorter.scala:193)\n\tat org.apache.spark.shuffle.sort.SortShuffleWriter.write(SortShuffleWriter.scala:63)\n\tat org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:96)\n\tat org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:109)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:345)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\n\tat java.lang.Thread.run(Thread.java:748)\n",
"taskMetrics" : {
"executorDeserializeTime" : 0,
"executorDeserializeCpuTime" : 0,
"executorRunTime" : 1332,
"executorCpuTime" : 0,
"resultSize" : 0,
"jvmGcTime" : 33,
"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" : 3075188,
"recordsWritten" : 0
}
}
},
"6" : {
"taskId" : 6,
"index" : 6,
"attempt" : 0,
"launchTime" : "2018-01-18T18:33:14.323GMT",
"duration" : 67,
"executorId" : "4",
"host" : "apiros-2.gce.test.com",
"status" : "FAILED",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"errorMessage" : "java.lang.RuntimeException: Bad executor\n\tat $line15.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$2.apply(<console>:28)\n\tat $line15.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$2.apply(<console>:27)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.collection.ExternalSorter.insertAll(ExternalSorter.scala:193)\n\tat org.apache.spark.shuffle.sort.SortShuffleWriter.write(SortShuffleWriter.scala:63)\n\tat org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:96)\n\tat org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:109)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:345)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\n\tat java.lang.Thread.run(Thread.java:748)\n",
"taskMetrics" : {
"executorDeserializeTime" : 0,
"executorDeserializeCpuTime" : 0,
"executorRunTime" : 51,
"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" : 183718,
"recordsWritten" : 0
}
}
},
"9" : {
"taskId" : 9,
"index" : 4,
"attempt" : 1,
"launchTime" : "2018-01-18T18:33:14.973GMT",
"duration" : 96,
"executorId" : "2",
"host" : "apiros-3.gce.test.com",
"status" : "SUCCESS",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"taskMetrics" : {
"executorDeserializeTime" : 5,
"executorDeserializeCpuTime" : 4793905,
"executorRunTime" : 48,
"executorCpuTime" : 25678331,
"resultSize" : 1091,
"jvmGcTime" : 0,
"resultSerializationTime" : 1,
"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" : 147,
"writeTime" : 366050,
"recordsWritten" : 3
}
}
},
"13" : {
"taskId" : 13,
"index" : 9,
"attempt" : 0,
"launchTime" : "2018-01-18T18:33:15.200GMT",
"duration" : 76,
"executorId" : "2",
"host" : "apiros-3.gce.test.com",
"status" : "SUCCESS",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"taskMetrics" : {
"executorDeserializeTime" : 25,
"executorDeserializeCpuTime" : 5860574,
"executorRunTime" : 25,
"executorCpuTime" : 20585619,
"resultSize" : 1048,
"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" : 147,
"writeTime" : 369513,
"recordsWritten" : 3
}
}
},
"2" : {
"taskId" : 2,
"index" : 2,
"attempt" : 0,
"launchTime" : "2018-01-18T18:33:12.832GMT",
"duration" : 1774,
"executorId" : "3",
"host" : "apiros-2.gce.test.com",
"status" : "SUCCESS",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"taskMetrics" : {
"executorDeserializeTime" : 1206,
"executorDeserializeCpuTime" : 263386625,
"executorRunTime" : 493,
"executorCpuTime" : 278399617,
"resultSize" : 1134,
"jvmGcTime" : 78,
"resultSerializationTime" : 1,
"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" : 144,
"writeTime" : 3322956,
"recordsWritten" : 3
}
}
},
"12" : {
"taskId" : 12,
"index" : 8,
"attempt" : 0,
"launchTime" : "2018-01-18T18:33:15.165GMT",
"duration" : 60,
"executorId" : "1",
"host" : "apiros-3.gce.test.com",
"status" : "SUCCESS",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"taskMetrics" : {
"executorDeserializeTime" : 4,
"executorDeserializeCpuTime" : 4010338,
"executorRunTime" : 34,
"executorCpuTime" : 21657558,
"resultSize" : 1048,
"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" : 147,
"writeTime" : 319101,
"recordsWritten" : 3
}
}
},
"7" : {
"taskId" : 7,
"index" : 5,
"attempt" : 1,
"launchTime" : "2018-01-18T18:33:14.859GMT",
"duration" : 115,
"executorId" : "2",
"host" : "apiros-3.gce.test.com",
"status" : "SUCCESS",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"taskMetrics" : {
"executorDeserializeTime" : 11,
"executorDeserializeCpuTime" : 10894331,
"executorRunTime" : 84,
"executorCpuTime" : 28283110,
"resultSize" : 1048,
"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" : 147,
"writeTime" : 377601,
"recordsWritten" : 3
}
}
},
"3" : {
"taskId" : 3,
"index" : 3,
"attempt" : 0,
"launchTime" : "2018-01-18T18:33:12.833GMT",
"duration" : 2027,
"executorId" : "2",
"host" : "apiros-3.gce.test.com",
"status" : "SUCCESS",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"taskMetrics" : {
"executorDeserializeTime" : 1282,
"executorDeserializeCpuTime" : 365807898,
"executorRunTime" : 681,
"executorCpuTime" : 349920830,
"resultSize" : 1134,
"jvmGcTime" : 102,
"resultSerializationTime" : 1,
"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" : 147,
"writeTime" : 3587839,
"recordsWritten" : 3
}
}
},
"11" : {
"taskId" : 11,
"index" : 7,
"attempt" : 0,
"launchTime" : "2018-01-18T18:33:15.072GMT",
"duration" : 93,
"executorId" : "1",
"host" : "apiros-3.gce.test.com",
"status" : "SUCCESS",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"taskMetrics" : {
"executorDeserializeTime" : 4,
"executorDeserializeCpuTime" : 4239884,
"executorRunTime" : 77,
"executorCpuTime" : 21689428,
"resultSize" : 1048,
"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" : 147,
"writeTime" : 323898,
"recordsWritten" : 3
}
}
},
"8" : {
"taskId" : 8,
"index" : 6,
"attempt" : 1,
"launchTime" : "2018-01-18T18:33:14.879GMT",
"duration" : 194,
"executorId" : "1",
"host" : "apiros-3.gce.test.com",
"status" : "SUCCESS",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"taskMetrics" : {
"executorDeserializeTime" : 56,
"executorDeserializeCpuTime" : 12246145,
"executorRunTime" : 54,
"executorCpuTime" : 27304550,
"resultSize" : 1048,
"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" : 147,
"writeTime" : 311940,
"recordsWritten" : 3
}
}
},
"4" : {
"taskId" : 4,
"index" : 4,
"attempt" : 0,
"launchTime" : "2018-01-18T18:33:12.833GMT",
"duration" : 1522,
"executorId" : "4",
"host" : "apiros-2.gce.test.com",
"status" : "FAILED",
"taskLocality" : "PROCESS_LOCAL",
"speculative" : false,
"accumulatorUpdates" : [ ],
"errorMessage" : "java.lang.RuntimeException: Bad executor\n\tat $line15.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$2.apply(<console>:28)\n\tat $line15.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$2.apply(<console>:27)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.collection.ExternalSorter.insertAll(ExternalSorter.scala:193)\n\tat org.apache.spark.shuffle.sort.SortShuffleWriter.write(SortShuffleWriter.scala:63)\n\tat org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:96)\n\tat org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:109)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:345)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\n\tat java.lang.Thread.run(Thread.java:748)\n",
"taskMetrics" : {
"executorDeserializeTime" : 0,
"executorDeserializeCpuTime" : 0,
"executorRunTime" : 1184,
"executorCpuTime" : 0,
"resultSize" : 0,
"jvmGcTime" : 82,
"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" : 16858066,
"recordsWritten" : 0
}
}
}
},
"executorSummary" : {
"4" : {
"taskTime" : 1589,
"failedTasks" : 2,
"succeededTasks" : 0,
"killedTasks" : 0,
"inputBytes" : 0,
"inputRecords" : 0,
"outputBytes" : 0,
"outputRecords" : 0,
"shuffleRead" : 0,
"shuffleReadRecords" : 0,
"shuffleWrite" : 0,
"shuffleWriteRecords" : 0,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"isBlacklistedForStage" : true
},
"5" : {
"taskTime" : 1579,
"failedTasks" : 2,
"succeededTasks" : 0,
"killedTasks" : 0,
"inputBytes" : 0,
"inputRecords" : 0,
"outputBytes" : 0,
"outputRecords" : 0,
"shuffleRead" : 0,
"shuffleReadRecords" : 0,
"shuffleWrite" : 0,
"shuffleWriteRecords" : 0,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"isBlacklistedForStage" : true
},
"1" : {
"taskTime" : 2411,
"failedTasks" : 0,
"succeededTasks" : 4,
"killedTasks" : 0,
"inputBytes" : 0,
"inputRecords" : 0,
"outputBytes" : 0,
"outputRecords" : 0,
"shuffleRead" : 0,
"shuffleReadRecords" : 0,
"shuffleWrite" : 585,
"shuffleWriteRecords" : 12,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"isBlacklistedForStage" : false
},
"2" : {
"taskTime" : 2446,
"failedTasks" : 0,
"succeededTasks" : 5,
"killedTasks" : 0,
"inputBytes" : 0,
"inputRecords" : 0,
"outputBytes" : 0,
"outputRecords" : 0,
"shuffleRead" : 0,
"shuffleReadRecords" : 0,
"shuffleWrite" : 732,
"shuffleWriteRecords" : 15,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"isBlacklistedForStage" : false
},
"3" : {
"taskTime" : 1774,
"failedTasks" : 0,
"succeededTasks" : 1,
"killedTasks" : 0,
"inputBytes" : 0,
"inputRecords" : 0,
"outputBytes" : 0,
"outputRecords" : 0,
"shuffleRead" : 0,
"shuffleReadRecords" : 0,
"shuffleWrite" : 144,
"shuffleWriteRecords" : 3,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0,
"isBlacklistedForStage" : true
}
},
"killedTasksSummary" : { }
}

View file

@ -1,4 +1,34 @@
[ {
"id" : "application_1516285256255_0012",
"name" : "Spark shell",
"attempts" : [ {
"startTime" : "2018-01-18T18:30:35.119GMT",
"endTime" : "2018-01-18T18:38:27.938GMT",
"lastUpdated" : "",
"duration" : 472819,
"sparkUser" : "attilapiros",
"completed" : true,
"appSparkVersion" : "2.3.0-SNAPSHOT",
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1516300235119,
"endTimeEpoch" : 1516300707938
} ]
}, {
"id" : "app-20180109111548-0000",
"name" : "Spark shell",
"attempts" : [ {
"startTime" : "2018-01-09T10:15:42.372GMT",
"endTime" : "2018-01-09T10:24:37.606GMT",
"lastUpdated" : "",
"duration" : 535234,
"sparkUser" : "attilapiros",
"completed" : true,
"appSparkVersion" : "2.3.0-SNAPSHOT",
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1515492942372,
"endTimeEpoch" : 1515493477606
} ]
}, {
"id" : "app-20161116163331-0000",
"name" : "Spark shell",
"attempts" : [ {
@ -9,9 +39,9 @@
"sparkUser" : "jose",
"completed" : true,
"appSparkVersion" : "2.1.0-SNAPSHOT",
"endTimeEpoch" : 1479335620587,
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1479335609916,
"lastUpdatedEpoch" : 0
"endTimeEpoch" : 1479335620587
} ]
}, {
"id" : "app-20161115172038-0000",
@ -24,9 +54,9 @@
"sparkUser" : "jose",
"completed" : true,
"appSparkVersion" : "2.1.0-SNAPSHOT",
"endTimeEpoch" : 1479252138874,
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1479252037079,
"lastUpdatedEpoch" : 0
"endTimeEpoch" : 1479252138874
} ]
}, {
"id" : "local-1430917381534",
@ -39,9 +69,9 @@
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "1.4.0-SNAPSHOT",
"endTimeEpoch" : 1430917391398,
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1430917380893,
"lastUpdatedEpoch" : 0
"endTimeEpoch" : 1430917391398
} ]
}, {
"id" : "local-1430917381535",
@ -55,9 +85,9 @@
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "1.4.0-SNAPSHOT",
"endTimeEpoch" : 1430917380950,
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1430917380893,
"lastUpdatedEpoch" : 0
"endTimeEpoch" : 1430917380950
}, {
"attemptId" : "1",
"startTime" : "2015-05-06T13:03:00.880GMT",
@ -67,9 +97,9 @@
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "1.4.0-SNAPSHOT",
"endTimeEpoch" : 1430917380890,
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1430917380880,
"lastUpdatedEpoch" : 0
"endTimeEpoch" : 1430917380890
} ]
}, {
"id" : "local-1426533911241",
@ -83,9 +113,9 @@
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "",
"endTimeEpoch" : 1426633945177,
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1426633910242,
"lastUpdatedEpoch" : 0
"endTimeEpoch" : 1426633945177
}, {
"attemptId" : "1",
"startTime" : "2015-03-16T19:25:10.242GMT",
@ -95,9 +125,9 @@
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "",
"endTimeEpoch" : 1426533945177,
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1426533910242,
"lastUpdatedEpoch" : 0
"endTimeEpoch" : 1426533945177
} ]
}, {
"id" : "local-1425081759269",
@ -110,10 +140,9 @@
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "",
"appSparkVersion" : "",
"endTimeEpoch" : 1425081766912,
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1425081758277,
"lastUpdatedEpoch" : 0
"endTimeEpoch" : 1425081766912
} ]
}, {
"id" : "local-1422981780767",
@ -126,9 +155,9 @@
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "",
"endTimeEpoch" : 1422981788731,
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1422981779720,
"lastUpdatedEpoch" : 0
"endTimeEpoch" : 1422981788731
} ]
}, {
"id" : "local-1422981759269",
@ -141,8 +170,8 @@
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "",
"endTimeEpoch" : 1422981766912,
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1422981758277,
"lastUpdatedEpoch" : 0
"endTimeEpoch" : 1422981766912
} ]
} ]

View file

@ -1,4 +1,34 @@
[ {
"id" : "application_1516285256255_0012",
"name" : "Spark shell",
"attempts" : [ {
"startTime" : "2018-01-18T18:30:35.119GMT",
"endTime" : "2018-01-18T18:38:27.938GMT",
"lastUpdated" : "",
"duration" : 472819,
"sparkUser" : "attilapiros",
"completed" : true,
"appSparkVersion" : "2.3.0-SNAPSHOT",
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1516300235119,
"endTimeEpoch" : 1516300707938
} ]
}, {
"id" : "app-20180109111548-0000",
"name" : "Spark shell",
"attempts" : [ {
"startTime" : "2018-01-09T10:15:42.372GMT",
"endTime" : "2018-01-09T10:24:37.606GMT",
"lastUpdated" : "",
"duration" : 535234,
"sparkUser" : "attilapiros",
"completed" : true,
"appSparkVersion" : "2.3.0-SNAPSHOT",
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1515492942372,
"endTimeEpoch" : 1515493477606
} ]
}, {
"id" : "app-20161116163331-0000",
"name" : "Spark shell",
"attempts" : [ {
@ -9,38 +39,8 @@
"sparkUser" : "jose",
"completed" : true,
"appSparkVersion" : "2.1.0-SNAPSHOT",
"endTimeEpoch" : 1479335620587,
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1479335609916,
"lastUpdatedEpoch" : 0
} ]
}, {
"id" : "app-20161115172038-0000",
"name" : "Spark shell",
"attempts" : [ {
"startTime" : "2016-11-15T23:20:37.079GMT",
"endTime" : "2016-11-15T23:22:18.874GMT",
"lastUpdated" : "",
"duration" : 101795,
"sparkUser" : "jose",
"completed" : true,
"appSparkVersion" : "2.1.0-SNAPSHOT",
"endTimeEpoch" : 1479252138874,
"startTimeEpoch" : 1479252037079,
"lastUpdatedEpoch" : 0
} ]
}, {
"id" : "local-1430917381534",
"name" : "Spark shell",
"attempts" : [ {
"startTime" : "2015-05-06T13:03:00.893GMT",
"endTime" : "2015-05-06T13:03:11.398GMT",
"lastUpdated" : "",
"duration" : 10505,
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "1.4.0-SNAPSHOT",
"endTimeEpoch" : 1430917391398,
"startTimeEpoch" : 1430917380893,
"lastUpdatedEpoch" : 0
"endTimeEpoch" : 1479335620587
} ]
} ]

View file

@ -1,4 +1,34 @@
[ {
"id" : "application_1516285256255_0012",
"name" : "Spark shell",
"attempts" : [ {
"startTime" : "2018-01-18T18:30:35.119GMT",
"endTime" : "2018-01-18T18:38:27.938GMT",
"lastUpdated" : "",
"duration" : 472819,
"sparkUser" : "attilapiros",
"completed" : true,
"appSparkVersion" : "2.3.0-SNAPSHOT",
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1516300235119,
"endTimeEpoch" : 1516300707938
} ]
}, {
"id" : "app-20180109111548-0000",
"name" : "Spark shell",
"attempts" : [ {
"startTime" : "2018-01-09T10:15:42.372GMT",
"endTime" : "2018-01-09T10:24:37.606GMT",
"lastUpdated" : "",
"duration" : 535234,
"sparkUser" : "attilapiros",
"completed" : true,
"appSparkVersion" : "2.3.0-SNAPSHOT",
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1515492942372,
"endTimeEpoch" : 1515493477606
} ]
}, {
"id" : "app-20161116163331-0000",
"name" : "Spark shell",
"attempts" : [ {
@ -9,9 +39,9 @@
"sparkUser" : "jose",
"completed" : true,
"appSparkVersion" : "2.1.0-SNAPSHOT",
"endTimeEpoch" : 1479335620587,
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1479335609916,
"lastUpdatedEpoch" : 0
"endTimeEpoch" : 1479335620587
} ]
}, {
"id" : "app-20161115172038-0000",
@ -24,9 +54,9 @@
"sparkUser" : "jose",
"completed" : true,
"appSparkVersion" : "2.1.0-SNAPSHOT",
"endTimeEpoch" : 1479252138874,
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1479252037079,
"lastUpdatedEpoch" : 0
"endTimeEpoch" : 1479252138874
} ]
}, {
"id" : "local-1430917381534",
@ -39,9 +69,9 @@
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "1.4.0-SNAPSHOT",
"endTimeEpoch" : 1430917391398,
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1430917380893,
"lastUpdatedEpoch" : 0
"endTimeEpoch" : 1430917391398
} ]
}, {
"id" : "local-1430917381535",
@ -55,9 +85,9 @@
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "1.4.0-SNAPSHOT",
"endTimeEpoch" : 1430917380950,
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1430917380893,
"lastUpdatedEpoch" : 0
"endTimeEpoch" : 1430917380950
}, {
"attemptId" : "1",
"startTime" : "2015-05-06T13:03:00.880GMT",
@ -67,9 +97,9 @@
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "1.4.0-SNAPSHOT",
"endTimeEpoch" : 1430917380890,
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1430917380880,
"lastUpdatedEpoch" : 0
"endTimeEpoch" : 1430917380890
} ]
}, {
"id" : "local-1426533911241",
@ -83,9 +113,9 @@
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "",
"endTimeEpoch" : 1426633945177,
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1426633910242,
"lastUpdatedEpoch" : 0
"endTimeEpoch" : 1426633945177
}, {
"attemptId" : "1",
"startTime" : "2015-03-16T19:25:10.242GMT",
@ -95,9 +125,9 @@
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "",
"endTimeEpoch" : 1426533945177,
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1426533910242,
"lastUpdatedEpoch" : 0
"endTimeEpoch" : 1426533945177
} ]
}, {
"id" : "local-1425081759269",
@ -110,8 +140,8 @@
"sparkUser" : "irashid",
"completed" : true,
"appSparkVersion" : "",
"endTimeEpoch" : 1425081766912,
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1425081758277,
"lastUpdatedEpoch" : 0
"endTimeEpoch" : 1425081766912
} ]
} ]

View file

@ -1,4 +1,34 @@
[ {
"id" : "application_1516285256255_0012",
"name" : "Spark shell",
"attempts" : [ {
"startTime" : "2018-01-18T18:30:35.119GMT",
"endTime" : "2018-01-18T18:38:27.938GMT",
"lastUpdated" : "",
"duration" : 472819,
"sparkUser" : "attilapiros",
"completed" : true,
"appSparkVersion" : "2.3.0-SNAPSHOT",
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1516300235119,
"endTimeEpoch" : 1516300707938
} ]
}, {
"id" : "app-20180109111548-0000",
"name" : "Spark shell",
"attempts" : [ {
"startTime" : "2018-01-09T10:15:42.372GMT",
"endTime" : "2018-01-09T10:24:37.606GMT",
"lastUpdated" : "",
"duration" : 535234,
"sparkUser" : "attilapiros",
"completed" : true,
"appSparkVersion" : "2.3.0-SNAPSHOT",
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1515492942372,
"endTimeEpoch" : 1515493477606
} ]
}, {
"id" : "app-20161116163331-0000",
"name" : "Spark shell",
"attempts" : [ {
@ -9,8 +39,8 @@
"sparkUser" : "jose",
"completed" : true,
"appSparkVersion" : "2.1.0-SNAPSHOT",
"startTimeEpoch" : 1479335609916,
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1479335609916,
"endTimeEpoch" : 1479335620587
} ]
}, {
@ -24,8 +54,8 @@
"sparkUser" : "jose",
"completed" : true,
"appSparkVersion" : "2.1.0-SNAPSHOT",
"startTimeEpoch" : 1479252037079,
"lastUpdatedEpoch" : 0,
"startTimeEpoch" : 1479252037079,
"endTimeEpoch" : 1479252138874
} ]
}, {

View file

@ -421,7 +421,8 @@
"shuffleWrite" : 13180,
"shuffleWriteRecords" : 0,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0
"diskBytesSpilled" : 0,
"isBlacklistedForStage" : false
}
},
"killedTasksSummary" : { }

View file

@ -421,7 +421,8 @@
"shuffleWrite" : 13180,
"shuffleWriteRecords" : 0,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0
"diskBytesSpilled" : 0,
"isBlacklistedForStage" : false
}
},
"killedTasksSummary" : { }

View file

@ -465,7 +465,8 @@
"shuffleWrite" : 0,
"shuffleWriteRecords" : 0,
"memoryBytesSpilled" : 0,
"diskBytesSpilled" : 0
"diskBytesSpilled" : 0,
"isBlacklistedForStage" : false
}
},
"killedTasksSummary" : { }

File diff suppressed because one or more lines are too long

File diff suppressed because one or more lines are too long

View file

@ -156,6 +156,8 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers
"applications/local-1426533911241/1/stages/0/0/taskList",
"stage task list from multi-attempt app json(2)" ->
"applications/local-1426533911241/2/stages/0/0/taskList",
"blacklisting for stage" -> "applications/app-20180109111548-0000/stages/0/0",
"blacklisting node for stage" -> "applications/application_1516285256255_0012/stages/0/0",
"rdd list storage json" -> "applications/local-1422981780767/storage/rdd",
"executor node blacklisting" -> "applications/app-20161116163331-0000/executors",

View file

@ -92,7 +92,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M
}
def createTaskSetBlacklist(stageId: Int = 0): TaskSetBlacklist = {
new TaskSetBlacklist(conf, stageId, clock)
new TaskSetBlacklist(listenerBusMock, conf, stageId, stageAttemptId = 0, clock = clock)
}
test("executors can be blacklisted with only a few failures per stage") {

View file

@ -16,18 +16,32 @@
*/
package org.apache.spark.scheduler
import org.mockito.Matchers.isA
import org.mockito.Mockito.{never, verify}
import org.scalatest.BeforeAndAfterEach
import org.scalatest.mockito.MockitoSugar
import org.apache.spark.{SparkConf, SparkFunSuite}
import org.apache.spark.internal.config
import org.apache.spark.util.{ManualClock, SystemClock}
import org.apache.spark.util.ManualClock
class TaskSetBlacklistSuite extends SparkFunSuite {
class TaskSetBlacklistSuite extends SparkFunSuite with BeforeAndAfterEach with MockitoSugar {
private var listenerBusMock: LiveListenerBus = _
override def beforeEach(): Unit = {
listenerBusMock = mock[LiveListenerBus]
super.beforeEach()
}
test("Blacklisting tasks, executors, and nodes") {
val conf = new SparkConf().setAppName("test").setMaster("local")
.set(config.BLACKLIST_ENABLED.key, "true")
val clock = new ManualClock
val attemptId = 0
val taskSetBlacklist = new TaskSetBlacklist(
listenerBusMock, conf, stageId = 0, stageAttemptId = attemptId, clock = clock)
val taskSetBlacklist = new TaskSetBlacklist(conf, stageId = 0, clock = clock)
clock.setTime(0)
// We will mark task 0 & 1 failed on both executor 1 & 2.
// We should blacklist all executors on that host, for all tasks for the stage. Note the API
@ -46,27 +60,53 @@ class TaskSetBlacklistSuite extends SparkFunSuite {
val shouldBeBlacklisted = (executor == "exec1" && index == 0)
assert(taskSetBlacklist.isExecutorBlacklistedForTask(executor, index) === shouldBeBlacklisted)
}
assert(!taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec1"))
verify(listenerBusMock, never())
.post(isA(classOf[SparkListenerExecutorBlacklistedForStage]))
assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA"))
verify(listenerBusMock, never())
.post(isA(classOf[SparkListenerNodeBlacklistedForStage]))
// Mark task 1 failed on exec1 -- this pushes the executor into the blacklist
taskSetBlacklist.updateBlacklistForFailedTask(
"hostA", exec = "exec1", index = 1, failureReason = "testing")
assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec1"))
verify(listenerBusMock).post(
SparkListenerExecutorBlacklistedForStage(0, "exec1", 2, 0, attemptId))
assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA"))
verify(listenerBusMock, never())
.post(isA(classOf[SparkListenerNodeBlacklistedForStage]))
// Mark one task as failed on exec2 -- not enough for any further blacklisting yet.
taskSetBlacklist.updateBlacklistForFailedTask(
"hostA", exec = "exec2", index = 0, failureReason = "testing")
assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec1"))
assert(!taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec2"))
assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA"))
verify(listenerBusMock, never())
.post(isA(classOf[SparkListenerNodeBlacklistedForStage]))
// Mark another task as failed on exec2 -- now we blacklist exec2, which also leads to
// blacklisting the entire node.
taskSetBlacklist.updateBlacklistForFailedTask(
"hostA", exec = "exec2", index = 1, failureReason = "testing")
assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec1"))
assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec2"))
verify(listenerBusMock).post(
SparkListenerExecutorBlacklistedForStage(0, "exec2", 2, 0, attemptId))
assert(taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA"))
verify(listenerBusMock).post(
SparkListenerNodeBlacklistedForStage(0, "hostA", 2, 0, attemptId))
// Make sure the blacklist has the correct per-task && per-executor responses, over a wider
// range of inputs.
for {
@ -81,6 +121,10 @@ class TaskSetBlacklistSuite extends SparkFunSuite {
// intentional, it keeps it fast and is sufficient for usage in the scheduler.
taskSetBlacklist.isExecutorBlacklistedForTask(executor, index) === (badExec && badIndex))
assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet(executor) === badExec)
if (badExec) {
verify(listenerBusMock).post(
SparkListenerExecutorBlacklistedForStage(0, executor, 2, 0, attemptId))
}
}
}
assert(taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA"))
@ -110,7 +154,14 @@ class TaskSetBlacklistSuite extends SparkFunSuite {
.set(config.MAX_TASK_ATTEMPTS_PER_NODE, 3)
.set(config.MAX_FAILURES_PER_EXEC_STAGE, 2)
.set(config.MAX_FAILED_EXEC_PER_NODE_STAGE, 3)
val taskSetBlacklist = new TaskSetBlacklist(conf, stageId = 0, new SystemClock())
val clock = new ManualClock
val attemptId = 0
val taskSetBlacklist = new TaskSetBlacklist(
listenerBusMock, conf, stageId = 0, stageAttemptId = attemptId, clock = clock)
var time = 0
clock.setTime(time)
// Fail a task twice on hostA, exec:1
taskSetBlacklist.updateBlacklistForFailedTask(
"hostA", exec = "1", index = 0, failureReason = "testing")
@ -118,37 +169,75 @@ class TaskSetBlacklistSuite extends SparkFunSuite {
"hostA", exec = "1", index = 0, failureReason = "testing")
assert(taskSetBlacklist.isExecutorBlacklistedForTask("1", 0))
assert(!taskSetBlacklist.isNodeBlacklistedForTask("hostA", 0))
assert(!taskSetBlacklist.isExecutorBlacklistedForTaskSet("1"))
verify(listenerBusMock, never()).post(
SparkListenerExecutorBlacklistedForStage(time, "1", 2, 0, attemptId))
assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA"))
verify(listenerBusMock, never()).post(
SparkListenerNodeBlacklistedForStage(time, "hostA", 2, 0, attemptId))
// Fail the same task once more on hostA, exec:2
time += 1
clock.setTime(time)
taskSetBlacklist.updateBlacklistForFailedTask(
"hostA", exec = "2", index = 0, failureReason = "testing")
assert(taskSetBlacklist.isNodeBlacklistedForTask("hostA", 0))
assert(!taskSetBlacklist.isExecutorBlacklistedForTaskSet("2"))
verify(listenerBusMock, never()).post(
SparkListenerExecutorBlacklistedForStage(time, "2", 2, 0, attemptId))
assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA"))
verify(listenerBusMock, never()).post(
SparkListenerNodeBlacklistedForStage(time, "hostA", 2, 0, attemptId))
// Fail another task on hostA, exec:1. Now that executor has failures on two different tasks,
// so its blacklisted
time += 1
clock.setTime(time)
taskSetBlacklist.updateBlacklistForFailedTask(
"hostA", exec = "1", index = 1, failureReason = "testing")
assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("1"))
verify(listenerBusMock)
.post(SparkListenerExecutorBlacklistedForStage(time, "1", 2, 0, attemptId))
assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA"))
verify(listenerBusMock, never())
.post(isA(classOf[SparkListenerNodeBlacklistedForStage]))
// Fail a third task on hostA, exec:2, so that exec is blacklisted for the whole task set
time += 1
clock.setTime(time)
taskSetBlacklist.updateBlacklistForFailedTask(
"hostA", exec = "2", index = 2, failureReason = "testing")
assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("2"))
verify(listenerBusMock)
.post(SparkListenerExecutorBlacklistedForStage(time, "2", 2, 0, attemptId))
assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA"))
verify(listenerBusMock, never())
.post(isA(classOf[SparkListenerNodeBlacklistedForStage]))
// Fail a fourth & fifth task on hostA, exec:3. Now we've got three executors that are
// blacklisted for the taskset, so blacklist the whole node.
time += 1
clock.setTime(time)
taskSetBlacklist.updateBlacklistForFailedTask(
"hostA", exec = "3", index = 3, failureReason = "testing")
taskSetBlacklist.updateBlacklistForFailedTask(
"hostA", exec = "3", index = 4, failureReason = "testing")
assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("3"))
verify(listenerBusMock)
.post(SparkListenerExecutorBlacklistedForStage(time, "3", 2, 0, attemptId))
assert(taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA"))
verify(listenerBusMock).post(
SparkListenerNodeBlacklistedForStage(time, "hostA", 3, 0, attemptId))
}
test("only blacklist nodes for the task set when all the blacklisted executors are all on " +
@ -157,22 +246,42 @@ class TaskSetBlacklistSuite extends SparkFunSuite {
// lead to any node blacklisting
val conf = new SparkConf().setAppName("test").setMaster("local")
.set(config.BLACKLIST_ENABLED.key, "true")
val taskSetBlacklist = new TaskSetBlacklist(conf, stageId = 0, new SystemClock())
val clock = new ManualClock
val attemptId = 0
val taskSetBlacklist = new TaskSetBlacklist(
listenerBusMock, conf, stageId = 0, stageAttemptId = attemptId, clock = clock)
var time = 0
clock.setTime(time)
taskSetBlacklist.updateBlacklistForFailedTask(
"hostA", exec = "1", index = 0, failureReason = "testing")
taskSetBlacklist.updateBlacklistForFailedTask(
"hostA", exec = "1", index = 1, failureReason = "testing")
assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("1"))
assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA"))
assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("1"))
verify(listenerBusMock)
.post(SparkListenerExecutorBlacklistedForStage(time, "1", 2, 0, attemptId))
assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA"))
verify(listenerBusMock, never()).post(
SparkListenerNodeBlacklistedForStage(time, "hostA", 2, 0, attemptId))
time += 1
clock.setTime(time)
taskSetBlacklist.updateBlacklistForFailedTask(
"hostB", exec = "2", index = 0, failureReason = "testing")
taskSetBlacklist.updateBlacklistForFailedTask(
"hostB", exec = "2", index = 1, failureReason = "testing")
assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("1"))
assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("2"))
verify(listenerBusMock)
.post(SparkListenerExecutorBlacklistedForStage(time, "2", 2, 0, attemptId))
assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA"))
assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostB"))
verify(listenerBusMock, never())
.post(isA(classOf[SparkListenerNodeBlacklistedForStage]))
}
}

View file

@ -251,6 +251,49 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
}
}
// Blacklisting executor for stage
time += 1
listener.onExecutorBlacklistedForStage(SparkListenerExecutorBlacklistedForStage(
time = time,
executorId = execIds.head,
taskFailures = 2,
stageId = stages.head.stageId,
stageAttemptId = stages.head.attemptId))
val executorStageSummaryWrappers =
store.view(classOf[ExecutorStageSummaryWrapper]).index("stage")
.first(key(stages.head))
.last(key(stages.head))
.asScala.toSeq
assert(executorStageSummaryWrappers.nonEmpty)
executorStageSummaryWrappers.foreach { exec =>
// only the first executor is expected to be blacklisted
val expectedBlacklistedFlag = exec.executorId == execIds.head
assert(exec.info.isBlacklistedForStage === expectedBlacklistedFlag)
}
// Blacklisting node for stage
time += 1
listener.onNodeBlacklistedForStage(SparkListenerNodeBlacklistedForStage(
time = time,
hostId = "2.example.com", // this is where the second executor is hosted
executorFailures = 1,
stageId = stages.head.stageId,
stageAttemptId = stages.head.attemptId))
val executorStageSummaryWrappersForNode =
store.view(classOf[ExecutorStageSummaryWrapper]).index("stage")
.first(key(stages.head))
.last(key(stages.head))
.asScala.toSeq
assert(executorStageSummaryWrappersForNode.nonEmpty)
executorStageSummaryWrappersForNode.foreach { exec =>
// both executor is expected to be blacklisted
assert(exec.info.isBlacklistedForStage === true)
}
// Fail one of the tasks, re-start it.
time += 1
s1Tasks.head.markFinished(TaskState.FAILED, time)

View file

@ -73,8 +73,10 @@ logs
.*dependency-reduced-pom.xml
known_translations
json_expectation
app-20180109111548-0000
app-20161115172038-0000
app-20161116163331-0000
application_1516285256255_0012
local-1422981759269
local-1422981780767
local-1425081759269