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

Dataset complex types mapping to DataFrame (element nullability) mismatch

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Blocker
    • Resolution: Fixed
    • 1.6.1
    • 1.6.1, 2.0.0
    • SQL
    • None

    Description

      There seems to be a regression between 1.6.0 and 1.6.1 (snapshot build). By default a scala Seq[Double] is mapped by Spark as an ArrayType with nullable element

       |-- valuations: array (nullable = true)
       |    |-- element: double (containsNull = true)
      

      This could be read back to as a Dataset in Spark 1.6.0

          val df = sqlContext.table("valuations").as[Valuation]
      

      But with Spark 1.6.1 the same fails with

          val df = sqlContext.table("valuations").as[Valuation]
      
      org.apache.spark.sql.AnalysisException: cannot resolve 'cast(valuations as array<double>)' due to data type mismatch: cannot cast ArrayType(DoubleType,true) to ArrayType(DoubleType,false);
      

      Here's the classes I am using

      case class Valuation(tradeId : String,
                           counterparty: String,
                           nettingAgreement: String,
                           wrongWay: Boolean,
                           valuations : Seq[Double], /* one per scenario */
                           timeInterval: Int,
                           jobId: String)  /* used for hdfs partitioning */
      
      val vals : Seq[Valuation] = Seq()
      val valsDF = sqlContext.sparkContext.parallelize(vals).toDF
      valsDF.write.partitionBy("jobId").mode(SaveMode.Overwrite).saveAsTable("valuations")
      

      even the following gives the same result

      val valsDF = vals.toDS.toDF
      

      Attachments

        Activity

          People

            cloud_fan Wenchen Fan
            deenar Deenar Toraskar
            Votes:
            0 Vote for this issue
            Watchers:
            8 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: