You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "zhengruifeng (via GitHub)" <gi...@apache.org> on 2023/11/02 02:25:12 UTC

[PR] [WIP][CONNECT] Fix column resolution in DataFrame.drop [spark]

zhengruifeng opened a new pull request, #43632:
URL: https://github.com/apache/spark/pull/43632

   ### What changes were proposed in this pull request?
   Fix column resolution in DataFrame.drop
   
   
   ### Why are the changes needed?
   ```
   from pyspark.sql.functions import col
   
   # create first dataframe
   left_df = spark.createDataFrame([(1, 'a'), (2, 'b'), (3, 'c')], ['join_key', 'value1'])
   # create second dataframe
   right_df = spark.createDataFrame([(1, 'aa'), (2, 'bb'), (4, 'dd')], ['join_key', 'value2'])
   joined_df = left_df.join(right_df, on=left_df['join_key'] == right_df['join_key'], how='left')
   display(joined_df)
   cleaned_df = joined_df.drop(left_df['join_key'])
   display(cleaned_df) # error here
   
   JVM stacktrace:
   org.apache.spark.sql.AnalysisException: [AMBIGUOUS_REFERENCE] Reference `join_key` is ambiguous, could be: [`join_key`, `join_key`]. SQLSTATE: 42704
   	at org.apache.spark.sql.errors.QueryCompilationErrors$.ambiguousReferenceError(QueryCompilationErrors.scala:1957)
   	at org.apache.spark.sql.catalyst.expressions.package$AttributeSeq.resolve(package.scala:377)
   	at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:156)
   	at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveQuoted(LogicalPlan.scala:167)
   	at org.apache.spark.sql.Dataset.$anonfun$drop$4(Dataset.scala:3071)
   ```
   
   
   
   ### Does this PR introduce _any_ user-facing change?
   no
   
   
   ### How was this patch tested?
   added ut
   
   ### Was this patch authored or co-authored using generative AI tooling?
   no


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45770][SQL][PYTHON][CONNECT] Fix column resolution in `DataFrame.drop` [spark]

Posted by "zhengruifeng (via GitHub)" <gi...@apache.org>.
zhengruifeng commented on code in PR #43632:
URL: https://github.com/apache/spark/pull/43632#discussion_r1380919985


##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -3064,13 +3064,31 @@ class Dataset[T] private[sql](
    */
   @scala.annotation.varargs
   def drop(col: Column, cols: Column*): DataFrame = {
-    val allColumns = col +: cols
-    val expressions = (for (col <- allColumns) yield col match {
+    val expressions = (col +: cols).map {
       case Column(u: UnresolvedAttribute) =>
-        queryExecution.analyzed.resolveQuoted(

Review Comment:
   I think new plan `DropColumns` should be a better approach



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45770][SQL][PYTHON][CONNECT] Fix column resolution in `DataFrame.drop` [spark]

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #43632:
URL: https://github.com/apache/spark/pull/43632#discussion_r1382497615


##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -3064,13 +3064,31 @@ class Dataset[T] private[sql](
    */
   @scala.annotation.varargs
   def drop(col: Column, cols: Column*): DataFrame = {
-    val allColumns = col +: cols
-    val expressions = (for (col <- allColumns) yield col match {
+    val expressions = (col +: cols).map {
       case Column(u: UnresolvedAttribute) =>
-        queryExecution.analyzed.resolveQuoted(

Review Comment:
   What's the behavior of
   `dropped_1 = joined_df.select(left_df["value1"], right_df["join_key"], right_df["value2"])` ?
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45770][SQL][PYTHON][CONNECT] Fix column resolution in `DataFrame.drop` [spark]

Posted by "zhengruifeng (via GitHub)" <gi...@apache.org>.
zhengruifeng closed pull request #43632: [SPARK-45770][SQL][PYTHON][CONNECT] Fix column resolution in `DataFrame.drop`
URL: https://github.com/apache/spark/pull/43632


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [WIP][SQL][PYTHON][CONNECT] Fix column resolution in `DataFrame.drop` [spark]

Posted by "zhengruifeng (via GitHub)" <gi...@apache.org>.
zhengruifeng commented on PR #43632:
URL: https://github.com/apache/spark/pull/43632#issuecomment-1790509377

   cc  @cloud-fan @ueshin @HyukjinKwon


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45770][SQL][PYTHON][CONNECT] Fix column resolution in `DataFrame.drop` [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #43632:
URL: https://github.com/apache/spark/pull/43632#discussion_r1380089299


##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -3064,13 +3064,31 @@ class Dataset[T] private[sql](
    */
   @scala.annotation.varargs
   def drop(col: Column, cols: Column*): DataFrame = {
-    val allColumns = col +: cols
-    val expressions = (for (col <- allColumns) yield col match {
+    val expressions = (col +: cols).map {
       case Column(u: UnresolvedAttribute) =>
-        queryExecution.analyzed.resolveQuoted(

Review Comment:
   I think the problem is we put complicated resolution logic in DF APIs. Shall we add a new logical plan `DropColumns`? It will be rewritten to `Project` when the columns are all resolved, so that we can reuse all the column resolution logic we implemented in the analyzer.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org