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

Liang-Chi Hsieh commented on SPARK-22335:
-----------------------------------------

[~CBribiescas] The column position in the schema of a Dataset doesn't 
necessarily match the fields in the typed objects. The document of {{as}} has 
explained it:

{code}
Returns a new Dataset where each record has been mapped on to the specified 
type. The
method used to map columns depend on the type of `U`:
 - When `U` is a class, fields for the class will be mapped to columns of the 
same name
   (case sensitivity is determined by `spark.sql.caseSensitive`).
{code}

{{unionByName}} resolves columns by name. For the typed objects, I think it can 
satisfy this kind of usage.

I remember that this is not the first ticket opened for union behavior on 
Datasets. The first thing might be that we can better describe this in the 
document of {{union}}.



> Union for DataSet uses column order instead of types for union
> --------------------------------------------------------------
>
>                 Key: SPARK-22335
>                 URL: https://issues.apache.org/jira/browse/SPARK-22335
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 2.2.0
>            Reporter: Carlos Bribiescas
>
> I see union uses column order for a DF. This to me is "fine" since they 
> aren't typed.
> However, for a dataset which is supposed to be strongly typed it is actually 
> giving the wrong result. If you try to access the members by name, it will 
> use the order. Heres is a reproducible case. 2.2.0
> {code:java}
>   case class AB(a : String, b : String)
>   val abDf = sc.parallelize(List(("aThing","bThing"))).toDF("a", "b")
>   val baDf = sc.parallelize(List(("bThing","aThing"))).toDF("b", "a")
>   
>   abDf.union(baDf).show() // as linked ticket states, its "Not a problem"
>   
>   val abDs = abDf.as[AB]
>   val baDs = baDf.as[AB]
>   
>   abDs.union(baDs).show()  // This gives wrong result since a Dataset[AB] 
> should be correctly mapped by type, not by column order
>   
>   abDs.union(baDs).map(_.a).show() // This gives wrong result since a 
> Dataset[AB] should be correctly mapped by type, not by column order
>    abDs.union(baDs).rdd.take(2) // This also gives wrong result
>   baDs.map(_.a).show() // However, this gives the correct result, even though 
> columns were out of order.
>   abDs.map(_.a).show() // This is correct too
>   baDs.select("a","b").as[AB].union(abDs).show() // This is the same 
> workaround for linked issue, slightly modified.  However this seems wrong 
> since its supposed to be strongly typed
>   
>   baDs.rdd.toDF().as[AB].union(abDs).show()  // This however gives correct 
> result, which is logically inconsistent behavior
>   abDs.rdd.union(baDs.rdd).toDF().show() // Simpler example that gives 
> correct result
> {code}
> So its inconsistent and a bug IMO.  And I'm not sure that the suggested work 
> around is really fair, since I'm supposed to be getting of type `AB`.  More 
> importantly I think the issue is bigger when you consider that it happens 
> even if you read from parquet (as you would expect).  And that its 
> inconsistent when going to/from rdd.
> I imagine its just lazily converting to typed DS instead of initially.  So 
> either that typing could be prioritized to happen before the union or 
> unioning of DF could be done with column order taken into account.  Again, 
> this is speculation..



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to