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

Scala value classes create encoder problems and break at runtime

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 1.6.2, 2.0.0
    • 2.1.0
    • Spark Core, SQL
    • None
    • JDK 8 on MacOS
      Scala 2.11.8
      Spark 2.0.0

    Description

      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.

      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)
      

      Test code for Spark 2.0.0:

      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}")
        }
      }
      
      

      Attachments

        Issue Links

          Activity

            People

              jodersky Jakob Odersky
              arisofalaska@gmail.com Aris Vlasakakis
              Votes:
              0 Vote for this issue
              Watchers:
              7 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: