Skip to content

Conversation

@zhengruifeng
Copy link
Contributor

@zhengruifeng zhengruifeng commented Sep 6, 2023

What changes were proposed in this pull request?

  • Make getitem work with duplicated columns
  • Disallow bool type index
  • Disallow negative index

Why are the changes needed?

1, SQL feature OrderBy ordinal works with duplicated columns

In [4]: df = spark.sql("SELECT * FROM VALUES (1, 1.1, 'a'), (2, 2.2, 'b'), (4, 4.4, 'c') AS TAB(a, a, a)")

In [5]: df.createOrReplaceTempView("v")

In [6]: spark.sql("SELECT * FROM v ORDER BY 1, 2").show()
+---+---+---+
|  a|  a|  a|
+---+---+---+
|  1|1.1|  a|
|  2|2.2|  b|
|  4|4.4|  c|
+---+---+---+

To support it in DataFame APIs, we need to make getitem work with duplicated columns

2 & 3: should be unintentional

Does this PR introduce any user-facing change?

YES

1, Make getitem work with duplicated columns
before

In [1]: df = spark.sql("SELECT * FROM VALUES (1, 1.1, 'a'), (2, 2.2, 'b'), (4, 4.4, 'c') AS TAB(a, a, a)")

In [2]: df[0]
---------------------------------------------------------------------------
AnalysisException                         Traceback (most recent call last)
Cell In[2], line 1
----> 1 df[0]
...

AnalysisException: [AMBIGUOUS_REFERENCE] Reference `a` is ambiguous, could be: [`TAB`.`a`, `TAB`.`a`, `TAB`.`a`].

In [3]: df[1]
---------------------------------------------------------------------------
AnalysisException                         Traceback (most recent call last)
...

AnalysisException: [AMBIGUOUS_REFERENCE] Reference `a` is ambiguous, could be: [`TAB`.`a`, `TAB`.`a`, `TAB`.`a`].

In [4]: df.orderBy(1, 2).show()
---------------------------------------------------------------------------
AnalysisException                         Traceback (most recent call last)
Cell In[7], line 1
----> 1 df.orderBy(1, 2).show()

...

AnalysisException: [AMBIGUOUS_REFERENCE] Reference `a` is ambiguous, could be: [`TAB`.`a`, `TAB`.`a`, `TAB`.`a`].

after

In [1]: df = spark.sql("SELECT * FROM VALUES (1, 1.1, 'a'), (2, 2.2, 'b'), (4, 4.4, 'c') AS TAB(a, a, a)")

In [2]: df[0]
Out[2]: Column<'a'>

In [3]: df[1]
Out[3]: Column<'a'>

In [4]: df.orderBy(1, 2).show()
+---+---+---+
|  a|  a|  a|
+---+---+---+
|  1|1.1|  a|
|  2|2.2|  b|
|  4|4.4|  c|
+---+---+---+

How was this patch tested?

added UTs

Was this patch authored or co-authored using generative AI tooling?

NO

Copy link
Member

@HyukjinKwon HyukjinKwon left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM otherwise

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

also cc @cloud-fan for the usage of GetColumnByOrdinal

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Just a question. Is this a relevant change?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it is not related, it is a not-used import. since we are touching this file, what about also removing it btw?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why don't we delete this? Is this comment required still?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nice, will remove it.

Copy link
Member

@dongjoon-hyun dongjoon-hyun left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1, LGTM (except two minor questions)

@zhengruifeng
Copy link
Contributor Author

merged to master, thanks @dongjoon-hyun and @HyukjinKwon for review


# accepted type and values
for index in [False, True, 0, 1, 2, -1, -2, -3]:
df[index]
Copy link
Contributor

@cloud-fan cloud-fan Sep 27, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is really a bad API. df.col can be ambiguous as people may use the column reference far away from the dataframe, e.g. df1.join(df2).select...filter...select(df1.col). We recommend users use qualified unresolved column instead, like col("t1.col"). Now df[index] is even worse as it only makes sense to use it immediately in current df's transformation.

Why do we add such an API? To support order by ordinal, we can just order by integer literals. The SQL parser also parses ORDER BY 1, 2 as ordering by integer literal 1 and 2, and analyzer will properly resolve it.

cc @HyukjinKwon @zhengruifeng

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If df[index] is already in pyspark for a while, I think it's fine to treat it as a shortcut of df.i_th_col. We shouldn't use GetColumnByOrdinal in this case, as it was added for Dataset Tuple encoding and it's guaranteed that we want to get the column from the direct child of the current plan node. But here, we can't guarantee this, as people can do df1.select..filter...groupBy...select(df1[index])

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

6183b5e

df[index] has been supported since spark 2.0.0.

To support df.groupBy(1, 2, 3) and df.orderBy(1, 2, 3), right now GetColumnByOrdinal is only used in the direct child internally.

The SQL parser also parses ORDER BY 1, 2 as ordering by integer literal 1 and 2, and analyzer will properly resolve it.

Do you mean use should directly SortOrder(UnresolvedOrdinal(index)) ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

have offline discussion with wenchen, will fix it by switching to SortOrder(Literal(index)). Will fix it next week.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

scala> val df = Seq((2, 1), (1, 2)).toDF("a", "b")
val df: org.apache.spark.sql.DataFrame = [a: int, b: int]

scala> df.show()
+---+---+
|  a|  b|
+---+---+
|  2|  1|
|  1|  2|
+---+---+


scala> df.orderBy(lit(1)).show()
+---+---+
|  a|  b|
+---+---+
|  1|  2|
|  2|  1|
+---+---+


scala> df.groupBy(lit(1)).agg(first(col("a")), max(col("b"))).show()
+---+--------+------+
|  1|first(a)|max(b)|
+---+--------+------+
|  1|       2|     2|
+---+--------+------+

it seems orderBy(lit(1)) directly works, while groupBy(lit(1)) needs some investigation.

Let me revert this PR first

zhengruifeng added a commit that referenced this pull request Oct 4, 2023
…cated column"

### What changes were proposed in this pull request?
This reverts commit 73d3c49.

### Why are the changes needed?
to address #42828 (comment) and #43115 (comment), should not use `GetColumnByOrdinal` in this case.

Need to find another approach, but let's revert it first.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
CI

### Was this patch authored or co-authored using generative AI tooling?
No

Closes #43211 from zhengruifeng/revert_SPARK_45088.

Authored-by: Ruifeng Zheng <[email protected]>
Signed-off-by: Ruifeng Zheng <[email protected]>
LuciferYang pushed a commit to LuciferYang/spark that referenced this pull request Oct 7, 2023
…cated column"

### What changes were proposed in this pull request?
This reverts commit 73d3c49.

### Why are the changes needed?
to address apache#42828 (comment) and apache#43115 (comment), should not use `GetColumnByOrdinal` in this case.

Need to find another approach, but let's revert it first.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
CI

### Was this patch authored or co-authored using generative AI tooling?
No

Closes apache#43211 from zhengruifeng/revert_SPARK_45088.

Authored-by: Ruifeng Zheng <[email protected]>
Signed-off-by: Ruifeng Zheng <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants