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

Cheng Lian edited comment on SPARK-13101 at 2/1/16 7:55 PM:
------------------------------------------------------------

I think this is expected behavior. The problem in snippet mentioned in the JIRA 
description is that it tries to convert a nullable DataFrame field into a 
non-nullable Dataset field, which shouldn't be allowed. This can be further 
illustrated by the following spark-shell snippet:
{code}
// Converting non-nullable DF field to nullable DS field, OK
case class Rec(a: Array[Integer])
val df = Seq(Array(0)).map(Tuple1(_)).toDF("a")
df.printSchema
df.as[Rec]

// Converting nullable DF field to non-nullable DS field, failure
case class Rec(a: Array[Int])
val df = Seq(Array(0: Integer)).map(Tuple1(_)).toDF("a")
df.printSchema
df.as[Rec]
{code}


was (Author: lian cheng):
I think this is expected behavior. The problem in snippet mentioned in the JIRA 
description is that it tries to convert a nullable DataFrame field into a 
non-nullable Dataset field, which shouldn't be allowed. This can be further 
illustrated by the following spark-shell snippet:
{code}
// Converting non-nullable DF field to nullable DS field, OK
case class Rec(a: Array[Integer])
val df = Seq(Array(0)).map(Tuple1(_)).toDF("a")
df.printSchema
df.as[Rec]

// Converting nullable DF field to non-nullable DS field, failure
case class Rec(a: Array[Integer])
val df = Seq(Array(0)).map(Tuple1(_)).toDF("a")
df.printSchema
df.as[Rec]
{code}

> Dataset complex types mapping to DataFrame  (element nullability) mismatch
> --------------------------------------------------------------------------
>
>                 Key: SPARK-13101
>                 URL: https://issues.apache.org/jira/browse/SPARK-13101
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 1.6.1
>            Reporter: Deenar Toraskar
>            Priority: Blocker
>
> 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
> {noformat}
>  |-- valuations: array (nullable = true)
>  |    |-- element: double (containsNull = true)
> {noformat}
> This could be read back to as a Dataset in Spark 1.6.0
> {code}
>     val df = sqlContext.table("valuations").as[Valuation]
> {code}
> But with Spark 1.6.1 the same fails with
> {code}
>     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);
> {code}
> Here's the classes I am using
> {code}
> 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")
> {code}
> even the following gives the same result
> {code}
> val valsDF = vals.toDS.toDF
> {code}



--
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

Reply via email to