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

Adrian Wang commented on SPARK-13393:
-------------------------------------

Hi [~srinathsmn]

In this `errorDF`, both `df('id')` and `varadha('id')` has the same 
`exprId`(they all come from `df`), so we cannot disambiguate between them in 
this design now.

As a workaround,  you should write code like `correctDF`, assign an alias for 
the columns first, or you can register df as table and then use a complete SQL 
query to get your data.

I think this is a bug under current design. I think we should put down the 
dataframe information in `Column` instances, and use an interval representation 
of `JoindeData` as the return value of `def join()`, in order to resolve 
ambiguity caused by self-join. For now, even I write something like

    val errorDF = df.join(varadha, df("id") === df("id"), 
"left_outer").select(df("id"), varadha("id") as "varadha_id")

The result would still be the same, since we are assuming condition with 
ambiguity should always be resolved to both sides.

I can draft a design doc for this if you are interested.
cc [~smilegator][~rxin][~marmbrus]

> Column mismatch issue in left_outer join using Spark DataFrame
> --------------------------------------------------------------
>
>                 Key: SPARK-13393
>                 URL: https://issues.apache.org/jira/browse/SPARK-13393
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 1.5.0
>            Reporter: Varadharajan
>
> Consider the below snippet:
> {code:title=test.scala|borderStyle=solid}
> case class Person(id: Int, name: String)
> val df = sc.parallelize(List(
>   Person(1, "varadha"),
>   Person(2, "nagaraj")
> )).toDF
> val varadha = df.filter("id = 1")
> val errorDF = df.join(varadha, df("id") === varadha("id"), 
> "left_outer").select(df("id"), varadha("id") as "varadha_id")
> val nagaraj = df.filter("id = 2").select(df("id") as "n_id")
> val correctDF = df.join(nagaraj, df("id") === nagaraj("n_id"), 
> "left_outer").select(df("id"), nagaraj("n_id") as "nagaraj_id")
> {code}
> The `errorDF` dataframe, after the left join is messed up and shows as below:
> | id|varadha_id|
> |  1|         1|
> |  2|         2 (*This should've been null*)| 
> whereas correctDF has the correct output after the left join:
> | id|nagaraj_id|
> |  1|      null|
> |  2|         2|



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