You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Bruce Robbins (JIRA)" <ji...@apache.org> on 2018/01/31 04:36:00 UTC

[jira] [Comment Edited] (SPARK-23251) ClassNotFoundException: scala.Any when there's a missing implicit Map encoder

    [ https://issues.apache.org/jira/browse/SPARK-23251?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16346246#comment-16346246 ] 

Bruce Robbins edited comment on SPARK-23251 at 1/31/18 4:35 AM:
----------------------------------------------------------------

[~srowen] This also occurs with compiled apps submitted via spark-submit.

For example, this app:
{code:java}
object Implicit1 {
 def main(args: Array[String]) {
 if (args.length < 1) {
 Console.err.println("No input file specified")
 System.exit(1)
 }
 val inputFilename = args(0)

 val spark = SparkSession.builder().appName("Implicit1").getOrCreate()
 import spark.implicits._

 val df = spark.read.json(inputFilename)
 //implicit val mapEncoder = org.apache.spark.sql.Encoders.kryo[Map[String, Any]]
 val results = df.map(row => row.getValuesMap[Any](List("stationName", "year"))).take(15)
 results.foreach(println)
 }
}{code}
When run on Spark 2.3 (via spark-submit), I get the same exception as I see with spark-shell.

With the implicit mapEncoder line uncommented, this compiles and runs fine on both 2.2 and 2.3.

Here's the exception from spark-submit on spark 2.3:

 
{noformat}
bash-3.2$ ./bin/spark-submit --version
 Welcome to
 ____ __
 / _/_ ___ ____/ /_
 \ \/ _ \/ _ `/ __/ '/
 /__/ ./_,// //_\ version 2.3.1-SNAPSHOT
 /_/
Using Scala version 2.11.8, Java HotSpot(TM) 64-Bit Server VM, 1.8.0_161
 Branch branch-2.3
 Compiled by user brobbins on 2018-01-28T01:25:18Z
 Revision 3b6fc286d105ae7de737c46e50cf941e6831ab98
 Url https://github.com/apache/spark.git
 Type --help for more information.
bash-3.2$ ./bin/spark-submit --class "Implicit1" ~/github/sparkAppPlay/target/scala-2.11/temps_2.11-1.0.jar ~/ncdc_gsod_short.jsonl 
 ......
 Exception in thread "main" java.lang.ClassNotFoundException: scala.Any
 at java.net.URLClassLoader.findClass(URLClassLoader.java:381)
 at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
 at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
 at java.lang.Class.forName0(Native Method)
 at java.lang.Class.forName(Class.java:348)
 at scala.reflect.runtime.JavaMirrors$JavaMirror.javaClass(JavaMirrors.scala:555)
 at scala.reflect.runtime.JavaMirrors$JavaMirror$$anonfun$classToJava$1.apply(JavaMirrors.scala:1211)
 at scala.reflect.runtime.JavaMirrors$JavaMirror$$anonfun$classToJava$1.apply(JavaMirrors.scala:1203)
 at scala.reflect.runtime.TwoWayCaches$TwoWayCache$$anonfun$toJava$1.apply(TwoWayCaches.scala:49)
 at scala.reflect.runtime.Gil$class.gilSynchronized(Gil.scala:19)
 at scala.reflect.runtime.JavaUniverse.gilSynchronized(JavaUniverse.scala:16)
 at scala.reflect.runtime.TwoWayCaches$TwoWayCache.toJava(TwoWayCaches.scala:44)
 at scala.reflect.runtime.JavaMirrors$JavaMirror.classToJava(JavaMirrors.scala:1203)
 at scala.reflect.runtime.JavaMirrors$JavaMirror.runtimeClass(JavaMirrors.scala:194)
 at scala.reflect.runtime.JavaMirrors$JavaMirror.runtimeClass(JavaMirrors.scala:54)
 at org.apache.spark.sql.catalyst.ScalaReflection$.getClassFromType(ScalaReflection.scala:700)
 at org.apache.spark.sql.catalyst.ScalaReflection$$anonfun$org$apache$spark$sql$catalyst$ScalaReflection$$dataTypeFor$1.apply(ScalaReflection.scala:84)
 at org.apache.spark.sql.catalyst.ScalaReflection$$anonfun$org$apache$spark$sql$catalyst$ScalaReflection$$dataTypeFor$1.apply(ScalaReflection.scala:65)
 at scala.reflect.internal.tpe.TypeConstraints$UndoLog.undo(TypeConstraints.scala:56)
 at org.apache.spark.sql.catalyst.ScalaReflection$class.cleanUpReflectionObjects(ScalaReflection.scala:824)
 at org.apache.spark.sql.catalyst.ScalaReflection$.cleanUpReflectionObjects(ScalaReflection.scala:39)
 at org.apache.spark.sql.catalyst.ScalaReflection$.org$apache$spark$sql$catalyst$ScalaReflection$$dataTypeFor(ScalaReflection.scala:64)
 at org.apache.spark.sql.catalyst.ScalaReflection$$anonfun$org$apache$spark$sql$catalyst$ScalaReflection$$serializerFor$1.apply(ScalaReflection.scala:512)
 at org.apache.spark.sql.catalyst.ScalaReflection$$anonfun$org$apache$spark$sql$catalyst$ScalaReflection$$serializerFor$1.apply(ScalaReflection.scala:445)
 at scala.reflect.internal.tpe.TypeConstraints$UndoLog.undo(TypeConstraints.scala:56)
 at org.apache.spark.sql.catalyst.ScalaReflection$class.cleanUpReflectionObjects(ScalaReflection.scala:824)
 at org.apache.spark.sql.catalyst.ScalaReflection$.cleanUpReflectionObjects(ScalaReflection.scala:39)
 at org.apache.spark.sql.catalyst.ScalaReflection$.org$apache$spark$sql$catalyst$ScalaReflection$$serializerFor(ScalaReflection.scala:445)
 at org.apache.spark.sql.catalyst.ScalaReflection$.serializerFor(ScalaReflection.scala:434)
 at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder$.apply(ExpressionEncoder.scala:71)
 at org.apache.spark.sql.SQLImplicits.newMapEncoder(SQLImplicits.scala:172)
 at Implicit1$.main(Implicit1.scala:17)
 at Implicit1.main(Implicit1.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.JavaMainApplication.start(SparkApplication.scala:52)
 at org.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:879)
 at org.apache.spark.deploy.SparkSubmit$.doRunMain$1(SparkSubmit.scala:197)
 at org.apache.spark.deploy.SparkSubmit$.submit(SparkSubmit.scala:227)
 at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:136)
 at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
 bash-3.2${noformat}
{noformat}
 


was (Author: bersprockets):
[~srowen] This also occurs with compiled apps submitted via spark-submit.

For example, this app:
{code:java}
object Implicit1 {
 def main(args: Array[String]) {
 if (args.length < 1) {
 Console.err.println("No input file specified")
 System.exit(1)
 }
 val inputFilename = args(0)

 val spark = SparkSession.builder().appName("Implicit1").getOrCreate()
 import spark.implicits._

 val df = spark.read.json(inputFilename)
 //implicit val mapEncoder = org.apache.spark.sql.Encoders.kryo[Map[String, Any]]
 val results = df.map(row => row.getValuesMap[Any](List("stationName", "year"))).take(15)
 results.foreach(println)
 }
}{code}
When run on Spark 2.3 (via spark-submit), I get the same exception as I see with spark-shell.

With the implicit mapEncoder line uncommented, this compiles and runs fine on both 2.2 and 2.3.

Here's the exception from spark-submit on spark 2.3:
{noformat}
bash-3.2$ ./bin/spark-submit --version
Welcome to
 ____ __
 / __/__ ___ _____/ /__
 _\ \/ _ \/ _ `/ __/ '_/
 /___/ .__/\_,_/_/ /_/\_\ version 2.3.1-SNAPSHOT
 /_/
 
Using Scala version 2.11.8, Java HotSpot(TM) 64-Bit Server VM, 1.8.0_161
Branch branch-2.3
Compiled by user brobbins on 2018-01-28T01:25:18Z
Revision 3b6fc286d105ae7de737c46e50cf941e6831ab98
Url https://github.com/apache/spark.git
Type --help for more information.

bash-3.2$ ./bin/spark-submit --class "Implicit1" ~/github/sparkAppPlay/target/scala-2.11/temps_2.11-1.0.jar ~/ncdc_gsod_short.jsonl 
......
Exception in thread "main" java.lang.ClassNotFoundException: scala.Any
 at java.net.URLClassLoader.findClass(URLClassLoader.java:381)
 at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
 at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
 at java.lang.Class.forName0(Native Method)
 at java.lang.Class.forName(Class.java:348)
 at scala.reflect.runtime.JavaMirrors$JavaMirror.javaClass(JavaMirrors.scala:555)
 at scala.reflect.runtime.JavaMirrors$JavaMirror$$anonfun$classToJava$1.apply(JavaMirrors.scala:1211)
 at scala.reflect.runtime.JavaMirrors$JavaMirror$$anonfun$classToJava$1.apply(JavaMirrors.scala:1203)
 at scala.reflect.runtime.TwoWayCaches$TwoWayCache$$anonfun$toJava$1.apply(TwoWayCaches.scala:49)
 at scala.reflect.runtime.Gil$class.gilSynchronized(Gil.scala:19)
 at scala.reflect.runtime.JavaUniverse.gilSynchronized(JavaUniverse.scala:16)
 at scala.reflect.runtime.TwoWayCaches$TwoWayCache.toJava(TwoWayCaches.scala:44)
 at scala.reflect.runtime.JavaMirrors$JavaMirror.classToJava(JavaMirrors.scala:1203)
 at scala.reflect.runtime.JavaMirrors$JavaMirror.runtimeClass(JavaMirrors.scala:194)
 at scala.reflect.runtime.JavaMirrors$JavaMirror.runtimeClass(JavaMirrors.scala:54)
 at org.apache.spark.sql.catalyst.ScalaReflection$.getClassFromType(ScalaReflection.scala:700)
 at org.apache.spark.sql.catalyst.ScalaReflection$$anonfun$org$apache$spark$sql$catalyst$ScalaReflection$$dataTypeFor$1.apply(ScalaReflection.scala:84)
 at org.apache.spark.sql.catalyst.ScalaReflection$$anonfun$org$apache$spark$sql$catalyst$ScalaReflection$$dataTypeFor$1.apply(ScalaReflection.scala:65)
 at scala.reflect.internal.tpe.TypeConstraints$UndoLog.undo(TypeConstraints.scala:56)
 at org.apache.spark.sql.catalyst.ScalaReflection$class.cleanUpReflectionObjects(ScalaReflection.scala:824)
 at org.apache.spark.sql.catalyst.ScalaReflection$.cleanUpReflectionObjects(ScalaReflection.scala:39)
 at org.apache.spark.sql.catalyst.ScalaReflection$.org$apache$spark$sql$catalyst$ScalaReflection$$dataTypeFor(ScalaReflection.scala:64)
 at org.apache.spark.sql.catalyst.ScalaReflection$$anonfun$org$apache$spark$sql$catalyst$ScalaReflection$$serializerFor$1.apply(ScalaReflection.scala:512)
 at org.apache.spark.sql.catalyst.ScalaReflection$$anonfun$org$apache$spark$sql$catalyst$ScalaReflection$$serializerFor$1.apply(ScalaReflection.scala:445)
 at scala.reflect.internal.tpe.TypeConstraints$UndoLog.undo(TypeConstraints.scala:56)
 at org.apache.spark.sql.catalyst.ScalaReflection$class.cleanUpReflectionObjects(ScalaReflection.scala:824)
 at org.apache.spark.sql.catalyst.ScalaReflection$.cleanUpReflectionObjects(ScalaReflection.scala:39)
 at org.apache.spark.sql.catalyst.ScalaReflection$.org$apache$spark$sql$catalyst$ScalaReflection$$serializerFor(ScalaReflection.scala:445)
 at org.apache.spark.sql.catalyst.ScalaReflection$.serializerFor(ScalaReflection.scala:434)
 at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder$.apply(ExpressionEncoder.scala:71)
 at org.apache.spark.sql.SQLImplicits.newMapEncoder(SQLImplicits.scala:172)
 at Implicit1$.main(Implicit1.scala:17)
 at Implicit1.main(Implicit1.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.JavaMainApplication.start(SparkApplication.scala:52)
 at org.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:879)
 at org.apache.spark.deploy.SparkSubmit$.doRunMain$1(SparkSubmit.scala:197)
 at org.apache.spark.deploy.SparkSubmit$.submit(SparkSubmit.scala:227)
 at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:136)
 at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
bash-3.2${noformat}

> ClassNotFoundException: scala.Any when there's a missing implicit Map encoder
> -----------------------------------------------------------------------------
>
>                 Key: SPARK-23251
>                 URL: https://issues.apache.org/jira/browse/SPARK-23251
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 2.3.1
>         Environment: mac os high sierra, centos 7
>            Reporter: Bruce Robbins
>            Priority: Minor
>
> In branch-2.2, when you attempt to use row.getValuesMap[Any] without an implicit Map encoder, you get a nice descriptive compile-time error:
> {noformat}
> scala> df.map(row => row.getValuesMap[Any](List("stationName", "year"))).collect
> <console>:26: error: Unable to find encoder for type stored in a Dataset.  Primitive types (Int, String, etc) and Product types (case classes) are supported by importing spark.implicits._  Support for serializing other types will be added in future releases.
>        df.map(row => row.getValuesMap[Any](List("stationName", "year"))).collect
>              ^
> scala> implicit val mapEncoder = org.apache.spark.sql.Encoders.kryo[Map[String, Any]]
> mapEncoder: org.apache.spark.sql.Encoder[Map[String,Any]] = class[value[0]: binary]
> scala> df.map(row => row.getValuesMap[Any](List("stationName", "year"))).collect
> res1: Array[Map[String,Any]] = Array(Map(stationName -> 007026 99999, year -> 2014), Map(stationName -> 007026 99999, year -> 2014), Map(stationName -> 007026 99999, year -> 2014),
> etc.......
> {noformat}
>  
>  On the latest master and also on branch-2.3, the transformation compiles (at least on spark-shell), but throws a ClassNotFoundException:
>  
> {noformat}
> scala> df.map(row => row.getValuesMap[Any](List("stationName", "year"))).collect
> java.lang.ClassNotFoundException: scala.Any
>  at scala.reflect.internal.util.AbstractFileClassLoader.findClass(AbstractFileClassLoader.scala:62)
>  at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
>  at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
>  at java.lang.Class.forName0(Native Method)
>  at java.lang.Class.forName(Class.java:348)
>  at scala.reflect.runtime.JavaMirrors$JavaMirror.javaClass(JavaMirrors.scala:555)
>  at scala.reflect.runtime.JavaMirrors$JavaMirror$$anonfun$classToJava$1.apply(JavaMirrors.scala:1211)
>  at scala.reflect.runtime.JavaMirrors$JavaMirror$$anonfun$classToJava$1.apply(JavaMirrors.scala:1203)
>  at scala.reflect.runtime.TwoWayCaches$TwoWayCache$$anonfun$toJava$1.apply(TwoWayCaches.scala:49)
>  at scala.reflect.runtime.Gil$class.gilSynchronized(Gil.scala:19)
>  at scala.reflect.runtime.JavaUniverse.gilSynchronized(JavaUniverse.scala:16)
>  at scala.reflect.runtime.TwoWayCaches$TwoWayCache.toJava(TwoWayCaches.scala:44)
>  at scala.reflect.runtime.JavaMirrors$JavaMirror.classToJava(JavaMirrors.scala:1203)
>  at scala.reflect.runtime.JavaMirrors$JavaMirror.runtimeClass(JavaMirrors.scala:194)
>  at scala.reflect.runtime.JavaMirrors$JavaMirror.runtimeClass(JavaMirrors.scala:54)
>  at org.apache.spark.sql.catalyst.ScalaReflection$.getClassFromType(ScalaReflection.scala:700)
>  at org.apache.spark.sql.catalyst.ScalaReflection$$anonfun$org$apache$spark$sql$catalyst$ScalaReflection$$dataTypeFor$1.apply(ScalaReflection.scala:84)
>  at org.apache.spark.sql.catalyst.ScalaReflection$$anonfun$org$apache$spark$sql$catalyst$ScalaReflection$$dataTypeFor$1.apply(ScalaReflection.scala:65)
>  at scala.reflect.internal.tpe.TypeConstraints$UndoLog.undo(TypeConstraints.scala:56)
>  at org.apache.spark.sql.catalyst.ScalaReflection$class.cleanUpReflectionObjects(ScalaReflection.scala:824)
>  at org.apache.spark.sql.catalyst.ScalaReflection$.cleanUpReflectionObjects(ScalaReflection.scala:39)
>  at org.apache.spark.sql.catalyst.ScalaReflection$.org$apache$spark$sql$catalyst$ScalaReflection$$dataTypeFor(ScalaReflection.scala:64)
>  at org.apache.spark.sql.catalyst.ScalaReflection$$anonfun$org$apache$spark$sql$catalyst$ScalaReflection$$serializerFor$1.apply(ScalaReflection.scala:512)
>  at org.apache.spark.sql.catalyst.ScalaReflection$$anonfun$org$apache$spark$sql$catalyst$ScalaReflection$$serializerFor$1.apply(ScalaReflection.scala:445)
>  at scala.reflect.internal.tpe.TypeConstraints$UndoLog.undo(TypeConstraints.scala:56)
>  at org.apache.spark.sql.catalyst.ScalaReflection$class.cleanUpReflectionObjects(ScalaReflection.scala:824)
>  at org.apache.spark.sql.catalyst.ScalaReflection$.cleanUpReflectionObjects(ScalaReflection.scala:39)
>  at org.apache.spark.sql.catalyst.ScalaReflection$.org$apache$spark$sql$catalyst$ScalaReflection$$serializerFor(ScalaReflection.scala:445)
>  at org.apache.spark.sql.catalyst.ScalaReflection$.serializerFor(ScalaReflection.scala:434)
>  at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder$.apply(ExpressionEncoder.scala:71)
>  at org.apache.spark.sql.SQLImplicits.newMapEncoder(SQLImplicits.scala:172)
>  ... 49 elided
> scala> implicit val mapEncoder = org.apache.spark.sql.Encoders.kryo[Map[String, Any]]
> mapEncoder: org.apache.spark.sql.Encoder[Map[String,Any]] = class[value[0]: binary]
> scala> df.map(row => row.getValuesMap[Any](List("stationName", "year"))).collect
> res1: Array[Map[String,Any]] = Array(Map(stationName -> 007026 99999, year -> 2014), Map(stationName -> 007026 99999, year -> 2014),
> etc.......
> {noformat}
>  
> This message is a lot less helpful.
> As with with 2.2, specifying the Map encoder allows the transformation and action to execute.
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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