GitHub user caneGuy opened a pull request:

    https://github.com/apache/spark/pull/19338

    [SPARK-21539][CORE] Add latest failure reason for task set blacklist

    ## What changes were proposed in this pull request?
    This patch add latest failure reason for task set blacklist.Which can be 
showed on spark ui and let user know failure reason directly.
    Till now , every job which aborted by completed blacklist just show log 
like below which has no more information:
    `Aborting $taskSet because task $indexInTaskSet (partition $partition) 
cannot run anywhere due to node and executor blacklist.  Blacklisting behavior 
cannot run anywhere due to node and executor blacklist.Blacklisting behavior 
can be configured via spark.blacklist.*."`
    **After modify:**
    `User class threw exception: org.apache.spark.SparkException: Job aborted 
due to stage failure: Aborting TaskSet 0.0 because task 0 (partition 0) cannot 
run anywhere due to node and executor blacklist. **Latest failure reason is** 
Some(Lost task 0.1 in stage 0.0 (TID 3,xxx, executor 1): java.lang.Exception: 
Fake error!
    at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:73)
    at org.apache.spark.scheduler.Task.run(Task.scala:99)
    at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:305)
    at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
    at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
    at java.lang.Thread.run(Thread.java:745)
    ). Blacklisting behavior can be configured via spark.blacklist.*.
    at 
org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1458)
    at 
org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1446)
    at 
org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1445)
    at 
scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
    at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
    at 
org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1445)
    at 
org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:808)
    at 
org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:808)
    at scala.Option.foreach(Option.scala:257)
    at 
org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:808)
    at 
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:1681)
    at 
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1636)
    at 
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1625)
    at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48)
    at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:634)
    at org.apache.spark.SparkContext.runJob(SparkContext.scala:1922)
    at org.apache.spark.SparkContext.runJob(SparkContext.scala:1935)
    at org.apache.spark.SparkContext.runJob(SparkContext.scala:1948)
    at org.apache.spark.SparkContext.runJob(SparkContext.scala:1962)
    at org.apache.spark.rdd.RDD.count(RDD.scala:1157)
    at org.apache.spark.examples.GroupByTest$.main(GroupByTest.scala:50)
    at org.apache.spark.examples.GroupByTest.main(GroupByTest.scala)
    at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
    at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
    at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
    at java.lang.reflect.Method.invoke(Method.java:498)
    at 
org.apache.spark.deploy.yarn.ApplicationMaster$$anon$2.run(ApplicationMaster.scala:653)`
    
    ## How was this patch tested?
    
    Unit test and manually test.


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/caneGuy/spark zhoukang/improve-blacklist

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/19338.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #19338
    
----
commit 668d5a72ca16170528336461999dad07ea77cada
Author: zhoukang <zhoukang199...@gmail.com>
Date:   2017-09-25T11:58:24Z

    Add latest failure reason for task set blacklist

----


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to