You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Daniel Darabos (JIRA)" <ji...@apache.org> on 2017/07/14 14:18:00 UTC

[jira] [Created] (SPARK-21418) NoSuchElementException: None.get on DataFrame.rdd

Daniel Darabos created SPARK-21418:
--------------------------------------

             Summary: NoSuchElementException: None.get on DataFrame.rdd
                 Key: SPARK-21418
                 URL: https://issues.apache.org/jira/browse/SPARK-21418
             Project: Spark
          Issue Type: Bug
          Components: SQL
    Affects Versions: 2.2.0
            Reporter: Daniel Darabos


I don't have a minimal reproducible example yet, sorry. I have the following lines in a unit test for our Spark application:

{code}
val df = mySparkSession.read.format("jdbc")
  .options(Map("url" -> url, "dbtable" -> "test_table"))
  .load()
df.show
println(df.rdd.collect)
{code}

The output shows the DataFrame contents from {{df.show}}. But the {{collect}} fails:

{noformat}
org.apache.spark.SparkException: Job aborted due to stage failure: Task serialization failed: java.util.NoSuchElementException: None.get
java.util.NoSuchElementException: None.get
  at scala.None$.get(Option.scala:347)
  at scala.None$.get(Option.scala:345)
  at org.apache.spark.sql.execution.DataSourceScanExec$class.org$apache$spark$sql$execution$DataSourceScanExec$$redact(DataSourceScanExec.scala:70)
  at org.apache.spark.sql.execution.DataSourceScanExec$$anonfun$4.apply(DataSourceScanExec.scala:54)
  at org.apache.spark.sql.execution.DataSourceScanExec$$anonfun$4.apply(DataSourceScanExec.scala:52)
  at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
  at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
  at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
  at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
  at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
  at scala.collection.AbstractTraversable.map(Traversable.scala:104)
  at org.apache.spark.sql.execution.DataSourceScanExec$class.simpleString(DataSourceScanExec.scala:52)
  at org.apache.spark.sql.execution.RowDataSourceScanExec.simpleString(DataSourceScanExec.scala:75)
  at org.apache.spark.sql.catalyst.plans.QueryPlan.verboseString(QueryPlan.scala:349)
  at org.apache.spark.sql.execution.RowDataSourceScanExec.org$apache$spark$sql$execution$DataSourceScanExec$$super$verboseString(DataSourceScanExec.scala:75)
  at org.apache.spark.sql.execution.DataSourceScanExec$class.verboseString(DataSourceScanExec.scala:60)
  at org.apache.spark.sql.execution.RowDataSourceScanExec.verboseString(DataSourceScanExec.scala:75)
  at org.apache.spark.sql.catalyst.trees.TreeNode.generateTreeString(TreeNode.scala:556)
  at org.apache.spark.sql.execution.WholeStageCodegenExec.generateTreeString(WholeStageCodegenExec.scala:451)
  at org.apache.spark.sql.catalyst.trees.TreeNode.generateTreeString(TreeNode.scala:576)
  at org.apache.spark.sql.catalyst.trees.TreeNode.treeString(TreeNode.scala:480)
  at org.apache.spark.sql.catalyst.trees.TreeNode.treeString(TreeNode.scala:477)
  at org.apache.spark.sql.catalyst.trees.TreeNode.toString(TreeNode.scala:474)
  at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1421)
  at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1178)
  at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1548)
  at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1509)
  at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1432)
  at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1178)
  at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1548)
  at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1509)
  at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1432)
  at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1178)
  at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1548)
  at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1509)
  at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1432)
  at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1178)
  at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1548)
  at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1509)
  at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1432)
  at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1178)
  at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1548)
  at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1509)
  at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1432)
  at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1178)
  at java.io.ObjectOutputStream.writeObject(ObjectOutputStream.java:348)
  at scala.collection.immutable.List$SerializationProxy.writeObject(List.scala:468)
  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 java.io.ObjectStreamClass.invokeWriteObject(ObjectStreamClass.java:1028)
  at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1496)
  at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1432)
  at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1178)
  at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1548)
  at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1509)
  at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1432)
  at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1178)
  at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1548)
  at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1509)
  at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1432)
  at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1178)
  at java.io.ObjectOutputStream.writeObject(ObjectOutputStream.java:348)
  at org.apache.spark.serializer.JavaSerializationStream.writeObject(JavaSerializer.scala:43)
  at org.apache.spark.serializer.JavaSerializerInstance.serialize(JavaSerializer.scala:100)
  at org.apache.spark.scheduler.DAGScheduler.submitMissingTasks(DAGScheduler.scala:1003)
  at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$submitStage(DAGScheduler.scala:930)
  at org.apache.spark.scheduler.DAGScheduler.handleJobSubmitted(DAGScheduler.scala:874)
  at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:1677)
  at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1669)
  at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1658)
  at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48)
{noformat}

Looks like this is due to https://github.com/apache/spark/commit/91fa80fe8a2480d64c430bd10f97b3d44c007bcc#diff-2a91a9a59953aa82fa132aaf45bd731bR69 from https://issues.apache.org/jira/browse/SPARK-20070. It tries to redact sensitive information from {{explain}} output. (We are not trying to explain anything here, so I doubt it is meant to be running in this case.) When it needs to access some configuration, it tries to take it from the "current" Spark session, which it just reads from a thread-local variable. We appear to be on a thread where this variable is not set.

My impression is that SPARK-20070 unintentionally introduced a very hard constraint on multi-threaded Spark applications.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org