You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Joseph Naegele (JIRA)" <ji...@apache.org> on 2016/09/08 14:47:20 UTC
[jira] [Commented] (SPARK-16599) java.util.NoSuchElementException:
None.get at at
org.apache.spark.storage.BlockInfoManager.releaseAllLocksForTask(BlockInfoManager.scala:343)
[ https://issues.apache.org/jira/browse/SPARK-16599?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15474055#comment-15474055 ]
Joseph Naegele commented on SPARK-16599:
----------------------------------------
I believe I can reproduce this. I have a singleton "Engine" object which instantiates a single {{SparkSession}} for my entire app.
Here is the relevant, *working* excerpt:
{code}
object Engine {
...
private var currentSession: SparkSession = null
def session: SparkSession = {
if (currentSession == null) {
Log.debug("Creating new SparkSession")
currentSession = SparkSession
.builder()
.appName(appName)
.master(Settings.SparkMaster)
.getOrCreate()
currentSession.udf.register(...)
}
currentSession
}
def someQuery(): Dataset[Foo] = {
val s = session
import s.implicits._
s.sql("select * from foo").as[Foo]
}
}
{code}
If I simplify it to the following, I see the reported error:
{code}
object Engine {
...
Log.debug("Creating new SparkSession")
val session = SparkSession
.builder()
.appName(appName)
.master(Settings.SparkMaster)
.getOrCreate()
session.udf.register(...)
def allFoos(): Dataset[Foo] = {
session.sql("select * from foo").as[Foo]
}
}
{code}
The exception occurs when I attempt to write a Dataset to disk (using spark-avro):
{code}
val foos = Engine.allFoos()
import com.databricks.spark.avro._
foos.write.mode(SaveMode.Overwrite).avro("hdfs:///foos")
{code}
Here's the stacktrace beginning from the {{write}} call above:
{code}
[Stage 2:> (0 + 2) / 7]16/09/08 14:24:39 WARN TaskSetManager: Lost task 2.0 in stage 2.0 (TID 5, 0.0.0.0): java.util.NoSuchElementException: None.get
at scala.None$.get(Option.scala:347)
at scala.None$.get(Option.scala:345)
at org.apache.spark.storage.BlockInfoManager.releaseAllLocksForTask(BlockInfoManager.scala:343)
at org.apache.spark.storage.BlockManager.releaseAllLocksForTask(BlockManager.scala:646)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:281)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:745)
[Stage 2:================> (2 + 2) / 7]16/09/08 14:24:39 ERROR TaskSetManager: Task 3 in stage 2.0 failed 4 times; aborting job
16/09/08 14:24:39 ERROR InsertIntoHadoopFsRelationCommand: Aborting job.
org.apache.spark.SparkException: Job aborted due to stage failure: Task 3 in stage 2.0 failed 4 times, most recent failure: Lost task 3.3 in stage 2.0 (TID 13, 0.0.0.0): java.util.NoSuchElementException: None.get
at scala.None$.get(Option.scala:347)
at scala.None$.get(Option.scala:345)
at org.apache.spark.storage.BlockInfoManager.releaseAllLocksForTask(BlockInfoManager.scala:343)
at org.apache.spark.storage.BlockManager.releaseAllLocksForTask(BlockManager.scala:646)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:281)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:745)
Driver stacktrace:
at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1429)
at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1417)
at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1416)
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:1416)
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:1640)
at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1599)
at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1588)
at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48)
at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:628)
at org.apache.spark.SparkContext.runJob(SparkContext.scala:1886)
at org.apache.spark.SparkContext.runJob(SparkContext.scala:1899)
at org.apache.spark.SparkContext.runJob(SparkContext.scala:1919)
at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand$$anonfun$run$1.apply$mcV$sp(InsertIntoHadoopFsRelationCommand.scala:143)
at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand$$anonfun$run$1.apply(InsertIntoHadoopFsRelationCommand.scala:115)
at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand$$anonfun$run$1.apply(InsertIntoHadoopFsRelationCommand.scala:115)
at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:57)
at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand.run(InsertIntoHadoopFsRelationCommand.scala:115)
at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:60)
at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:58)
at org.apache.spark.sql.execution.command.ExecutedCommandExec.doExecute(commands.scala:74)
at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:115)
at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:115)
at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:136)
at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:133)
at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:114)
at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:88)
at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:88)
at org.apache.spark.sql.execution.datasources.DataSource.write(DataSource.scala:495)
at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:213)
at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:196)
at com.databricks.spark.avro.package$AvroDataFrameWriter$$anonfun$avro$1.apply(package.scala:26)
at com.databricks.spark.avro.package$AvroDataFrameWriter$$anonfun$avro$1.apply(package.scala:26)
{code}
> java.util.NoSuchElementException: None.get at at org.apache.spark.storage.BlockInfoManager.releaseAllLocksForTask(BlockInfoManager.scala:343)
> ----------------------------------------------------------------------------------------------------------------------------------------------
>
> Key: SPARK-16599
> URL: https://issues.apache.org/jira/browse/SPARK-16599
> Project: Spark
> Issue Type: Bug
> Affects Versions: 2.0.0
> Environment: centos 6.7 spark 2.0
> Reporter: binde
>
> run a spark job with spark 2.0, error message
> Job aborted due to stage failure: Task 0 in stage 821.0 failed 4 times, most recent failure: Lost task 0.3 in stage 821.0 (TID 1480, e103): java.util.NoSuchElementException: None.get
> at scala.None$.get(Option.scala:347)
> at scala.None$.get(Option.scala:345)
> at org.apache.spark.storage.BlockInfoManager.releaseAllLocksForTask(BlockInfoManager.scala:343)
> at org.apache.spark.storage.BlockManager.releaseAllLocksForTask(BlockManager.scala:644)
> at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:281)
> 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)
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)
---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org