[ 
https://issues.apache.org/jira/browse/SPARK-20761?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Carlos Bribiescas updated SPARK-20761:
--------------------------------------
    Comment: was deleted

(was: This isn't quite the issue I'm facing, but solving this issue will fix my 
issue.  (probably)

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 this ticket states, its "Not a problem"
  
  val abDs = abDf.as[AB]
  val baDs = baDf.as[AB]
  
  abDs.union(baDs).show()
  
  abDs.union(baDs).map(_.a).show() // this gives wrong result since a 
Dataset[AB] should be correctly mapped by type, not by column order

  baDs.map(_.a).show() // However, this gives the correct result, even though 
columns were out of order.
  abDs.map(_.a).show() // So does this
{code}


So its inconsistent IMO.)

> Union uses column order rather than schema
> ------------------------------------------
>
>                 Key: SPARK-20761
>                 URL: https://issues.apache.org/jira/browse/SPARK-20761
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 2.1.1
>            Reporter: Nakul Jeirath
>            Priority: Minor
>
> I believe there is an issue when using union to combine two dataframes when 
> the order of columns differ between the left and right side of the union:
> {code}
> import org.apache.spark.sql.{Row, SparkSession}
> import org.apache.spark.sql.types.{BooleanType, StringType, StructField, 
> StructType}
> val schema = StructType(Seq(
>   StructField("id", StringType, false),
>   StructField("flag_one", BooleanType, false),
>   StructField("flag_two", BooleanType, false),
>   StructField("flag_three", BooleanType, false)
> ))
> val rowRdd = spark.sparkContext.parallelize(Seq(
>   Row("1", true, false, false),
>   Row("2", false, true, false),
>   Row("3", false, false, true)
> ))
> spark.createDataFrame(rowRdd, schema).createOrReplaceTempView("temp_flags")
> val emptyData = spark.createDataFrame(spark.sparkContext.emptyRDD[Row], 
> schema)
> //Select columns out of order with respect to the emptyData schema
> val data = emptyData.union(spark.sql("select id, flag_two, flag_three, 
> flag_one from temp_flags"))
> {code}
> Selecting the data from the "temp_flags" table results in:
> {noformat}
> spark.sql("select * from temp_flags").show
> +---+--------+--------+----------+
> | id|flag_one|flag_two|flag_three|
> +---+--------+--------+----------+
> |  1|    true|   false|     false|
> |  2|   false|    true|     false|
> |  3|   false|   false|      true|
> +---+--------+--------+----------+
> {noformat}
> Which is the data we'd expect but when inspecting "data" we get:
> {noformat}
> data.show()
> +---+--------+--------+----------+
> | id|flag_one|flag_two|flag_three|
> +---+--------+--------+----------+
> |  1|   false|   false|      true|
> |  2|    true|   false|     false|
> |  3|   false|    true|     false|
> +---+--------+--------+----------+
> {noformat}
> Having a non-empty dataframe on the left side of the union doesn't seem to 
> make a difference either:
> {noformat}
> spark.sql("select * from temp_flags").union(spark.sql("select id, flag_two, 
> flag_three, flag_one from temp_flags")).show
> +---+--------+--------+----------+
> | id|flag_one|flag_two|flag_three|
> +---+--------+--------+----------+
> |  1|    true|   false|     false|
> |  2|   false|    true|     false|
> |  3|   false|   false|      true|
> |  1|   false|   false|      true|
> |  2|    true|   false|     false|
> |  3|   false|    true|     false|
> +---+--------+--------+----------+
> {noformat}



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

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org

Reply via email to