Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-20945

NoSuchElementException key not found in TaskSchedulerImpl

    XMLWordPrintableJSON

    Details

    • Type: Bug
    • Status: Resolved
    • Priority: Minor
    • Resolution: Fixed
    • Affects Version/s: 2.1.0, 2.1.1
    • Fix Version/s: 2.3.0
    • Component/s: Scheduler
    • Labels:
      None
    • Flags:
      Important

      Description

      We started spark ThriftServer, however when a job fails it caused the thriftserver also shutted down.

      2017-05-27,10:34:27,272 INFO org.apache.spark.scheduler.cluster.YarnScheduler: Cancelling stage 21
      2017-05-27,10:34:27,274 ERROR org.apache.spark.scheduler.DAGSchedulerEventProcessLoop: DAGSchedulerEventProcessLoop failed; shutting down SparkContext
      java.util.NoSuchElementException: key not found: 14100
      at scala.collection.MapLike$class.default(MapLike.scala:228)
      at scala.collection.AbstractMap.default(Map.scala:59)
      at scala.collection.mutable.HashMap.apply(HashMap.scala:65)
      at org.apache.spark.scheduler.TaskSchedulerImpl$$anonfun$cancelTasks$2$$anonfun$apply$3$$anonfun$apply$1.apply$mcVJ$sp(TaskSchedulerImpl.scala:226)
      at org.apache.spark.scheduler.TaskSchedulerImpl$$anonfun$cancelTasks$2$$anonfun$apply$3$$anonfun$apply$1.apply(TaskSchedulerImpl.scala:225)
      at org.apache.spark.scheduler.TaskSchedulerImpl$$anonfun$cancelTasks$2$$anonfun$apply$3$$anonfun$apply$1.apply(TaskSchedulerImpl.scala:225)
      at scala.collection.mutable.HashSet.foreach(HashSet.scala:78)
      at org.apache.spark.scheduler.TaskSchedulerImpl$$anonfun$cancelTasks$2$$anonfun$apply$3.apply(TaskSchedulerImpl.scala:225)
      at org.apache.spark.scheduler.TaskSchedulerImpl$$anonfun$cancelTasks$2$$anonfun$apply$3.apply(TaskSchedulerImpl.scala:218)
      at scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:99)
      at scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:99)
      at scala.collection.mutable.HashTable$class.foreachEntry(HashTable.scala:230)
      at scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:40)
      at scala.collection.mutable.HashMap.foreach(HashMap.scala:99)
      at org.apache.spark.scheduler.TaskSchedulerImpl$$anonfun$cancelTasks$2.apply(TaskSchedulerImpl.scala:218)
      at org.apache.spark.scheduler.TaskSchedulerImpl$$anonfun$cancelTasks$2.apply(TaskSchedulerImpl.scala:217)
      at scala.Option.foreach(Option.scala:257)
      at org.apache.spark.scheduler.TaskSchedulerImpl.cancelTasks(TaskSchedulerImpl.scala:217)
      at org.apache.spark.scheduler.DAGScheduler$$anonfun$org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages$1.apply$mcVI$sp(DAGScheduler.scala:1461)
      at org.apache.spark.scheduler.DAGScheduler$$anonfun$org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages$1.apply(DAGScheduler.scala:1447)
      at org.apache.spark.scheduler.DAGScheduler$$anonfun$org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages$1.apply(DAGScheduler.scala:1447)
      at scala.collection.mutable.HashSet.foreach(HashSet.scala:78)
      at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1447)
      at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1423)
      at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1422)
      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:1422)
      at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:802)
      at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:802)
      at scala.Option.foreach(Option.scala:257)
      at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:802)
      at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:1650)
      at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1605)
      at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1594)
      at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48)

        Attachments

          Activity

            People

            • Assignee:
              liupengcheng liupengcheng
              Reporter:
              liupengcheng liupengcheng
            • Votes:
              0 Vote for this issue
              Watchers:
              4 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved: