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

Varun Shah commented on SPARK-37954:
------------------------------------

Hi [~andrewfmurphy], there are 2 reasons you dont see any error in the drop 
function:
 # As you rightly pointed out, the function will not throw any error in case 
the column does not exists. I think this is debatable, considering the fact 
that functions like select/filter throws error if column is missing.
 # The other factor is spark catalyst which optimizes your query/DAG by 
performing optimizations like predicate push down, which in the example you 
have mentioned tries pushing the filter downward and during this, recognizes 
the column rename and uses oldCol from the original dataframe/rdd
 # Run the following 2 examples to see how the Plans get created for different 
scenarios:

 
{code:java}
from pyspark.sql import SparkSession
from pyspark.sql.functions import col as col
spark = SparkSession.builder.appName('available_columns').getOrCreate()
df = spark.createDataFrame([{"oldcol": 1}, {"oldcol": 2}])
df = df.withColumnRenamed('oldcol', 'newcol')
df = df.filter(col("oldcol")!=2)
df.count()
df.explain("extended") {code}
 

 
{noformat}
== Parsed Logical Plan ==
'Filter NOT ('oldcol = 2)
+- Project [oldcol#168L AS newcol#170L]
   +- LogicalRDD [oldcol#168L], false

== Analyzed Logical Plan ==
newcol: bigint
Project [newcol#170L]
+- Filter NOT (oldcol#168L = cast(2 as bigint))
   +- Project [oldcol#168L AS newcol#170L, oldcol#168L]
      +- LogicalRDD [oldcol#168L], false

== Optimized Logical Plan ==
Project [oldcol#168L AS newcol#170L]
+- Filter (isnotnull(oldcol#168L) AND NOT (oldcol#168L = 2))
   +- LogicalRDD [oldcol#168L], false

== Physical Plan ==
*(1) Project [oldcol#168L AS newcol#170L]
+- *(1) Filter (isnotnull(oldcol#168L) AND NOT (oldcol#168L = 2))
   +- *(1) Scan ExistingRDD[oldcol#168L]{noformat}
 

--------------------------------------------------------------------------------------------------------------------------------

 
{code:java}
# action -2 
df2 = df.select(col("newcol"))
df2 = df2.filter(col("oldcol")!=2)
df2.count()
df2.explain("extended"){code}
 
{code:java}
== Parsed Logical Plan ==
'Filter NOT ('oldcol = 2)
+- Project [newcol#170L]
   +- Project [newcol#170L]
      +- Filter NOT (oldcol#168L = cast(2 as bigint))
         +- Project [oldcol#168L AS newcol#170L, oldcol#168L]
            +- LogicalRDD [oldcol#168L], false

== Analyzed Logical Plan ==
newcol: bigint
Project [newcol#170L]
+- Filter NOT (oldcol#168L = cast(2 as bigint))
   +- Project [newcol#170L, oldcol#168L]
      +- Project [newcol#170L, oldcol#168L]
         +- Filter NOT (oldcol#168L = cast(2 as bigint))
            +- Project [oldcol#168L AS newcol#170L, oldcol#168L]
               +- LogicalRDD [oldcol#168L], false

== Optimized Logical Plan ==
Project [oldcol#168L AS newcol#170L]
+- Filter (isnotnull(oldcol#168L) AND NOT (oldcol#168L = 2))
   +- LogicalRDD [oldcol#168L], false

== Physical Plan ==
*(1) Project [oldcol#168L AS newcol#170L]
+- *(1) Filter (isnotnull(oldcol#168L) AND NOT (oldcol#168L = 2))
   +- *(1) Scan ExistingRDD[oldcol#168L]


{code}
 

 

> old columns should not be available after select or drop
> --------------------------------------------------------
>
>                 Key: SPARK-37954
>                 URL: https://issues.apache.org/jira/browse/SPARK-37954
>             Project: Spark
>          Issue Type: Bug
>          Components: PySpark, SQL
>    Affects Versions: 3.0.1
>            Reporter: Jean Bon
>            Priority: Major
>
>  
> {code:java}
> from pyspark.sql import SparkSession
> from pyspark.sql.functions import col as col
> spark = SparkSession.builder.appName('available_columns').getOrCreate()
> df = spark.range(5).select((col("id")+10).alias("id2"))
> assert df.columns==["id2"] #OK
> try:
>     df.select("id")
>     error_raise = False
> except:
>     error_raise = True
> assert error_raise #OK
> df = df.drop("id") #should raise an error
> df.filter(col("id")!=2).count() #returns 4, should raise an error
> {code}
>  



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

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

Reply via email to