[ https://issues.apache.org/jira/browse/SPARK-17368?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15469564#comment-15469564 ]
Aris Vlasakakis commented on SPARK-17368: ----------------------------------------- It goes from inconvenient to actually prohibitive in a practical sense. I have a Dataset[Something], and inside case class Something I have various other case classes, and somewhere inside there there is a particular value class. It is so crazy to do manual unwrapping and rewrapping that at this point I just decided to eat the performance cost and use a regular class, not value class (I removed the 'extends AnyVal'). More generally, specially accommodating for value classes is *really hard* in a practical setting because if I have a whole bunch of ADTs and other case classes I'm working with, how do I know if anywhere in my domain I used a *value class* and I suddenly have to jump through a bunch of hoops just so Spark doesn't blow up? If I just had a Dataset[ThisIsAValueClass] with the top-level class being a value class, what you're saying is easy, but in practice the value class is one of many things somewhere deeper. > Scala value classes create encoder problems and break at runtime > ---------------------------------------------------------------- > > Key: SPARK-17368 > URL: https://issues.apache.org/jira/browse/SPARK-17368 > Project: Spark > Issue Type: Bug > Components: Spark Core, SQL > Affects Versions: 1.6.2, 2.0.0 > Environment: JDK 8 on MacOS > Scala 2.11.8 > Spark 2.0.0 > Reporter: Aris Vlasakakis > > Using Scala value classes as the inner type for Datasets breaks in Spark 2.0 > and 1.6.X. > This simple Spark 2 application demonstrates that the code will compile, but > will break at runtime with the error. The value class is of course > *FeatureId*, as it extends AnyVal. > {noformat} > Exception in thread "main" java.lang.RuntimeException: Error while encoding: > java.lang.RuntimeException: Couldn't find v on int > assertnotnull(input[0, int, true], top level non-flat input object).v AS v#0 > +- assertnotnull(input[0, int, true], top level non-flat input object).v > +- assertnotnull(input[0, int, true], top level non-flat input object) > +- input[0, int, true]". > at > org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.toRow(ExpressionEncoder.scala:279) > at > org.apache.spark.sql.SparkSession$$anonfun$3.apply(SparkSession.scala:421) > at > org.apache.spark.sql.SparkSession$$anonfun$3.apply(SparkSession.scala:421) > {noformat} > Test code for Spark 2.0.0: > {noformat} > import org.apache.spark.sql.{Dataset, SparkSession} > object BreakSpark { > case class FeatureId(v: Int) extends AnyVal > def main(args: Array[String]): Unit = { > val seq = Seq(FeatureId(1), FeatureId(2), FeatureId(3)) > val spark = SparkSession.builder.getOrCreate() > import spark.implicits._ > spark.sparkContext.setLogLevel("warn") > val ds: Dataset[FeatureId] = spark.createDataset(seq) > println(s"BREAK HERE: ${ds.count}") > } > } > {noformat} -- This message was sent by Atlassian JIRA (v6.3.4#6332) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org