[jira] [Commented] (SPARK-30421) Dropped columns still available for filtering
[ https://issues.apache.org/jira/browse/SPARK-30421?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17802355#comment-17802355 ] Maytas Monsereenusorn commented on SPARK-30421: --- Not sure if this is the right place to ask this but seems like there are certain cases where the column will not be available for filtering. This is also a regression from 2.1 to post 2.1. Example query: SELECT * FROM (SELECT x FROM (SELECT 1 AS x, 2 AS Y)) WHERE Y = '2' This works fine in 2.1 as for the reasons mentioned in this thread (due to _ResolveMissingReferences)_ However, after 2.1, the plan changed and SubqueryAlias was added. This seems to prevent ResolveMissingReferences from being able to change the project to add the Y column reference. In Post Spark 2.1 (i.e. Spark 3.3): {code:java} spark-sql-3.3> SELECT * FROM (SELECT x FROM (SELECT 1 AS x, 2 AS Y)) WHERE Y = '2'; Error in query: Column 'Y' does not exist. Did you mean one of the following? [__auto_generated_subquery_name.x]; line 1 pos 60; 'Project [*] +- 'Filter ('Y = 2) +- SubqueryAlias __auto_generated_subquery_name +- Project [x#30] +- SubqueryAlias __auto_generated_subquery_name +- Project [1 AS x#30, 2 AS Y#31] +- OneRowRelation{code} * Spark 2.1: {code:java} spark-sql> SELECT * FROM (SELECT x FROM (SELECT 1 AS x, 2 AS Y)) WHERE Y = '2'; 1 Time taken: 2.725 seconds, Fetched 1 row(s) spark-sql> EXPLAIN EXTENDED SELECT * FROM (SELECT x FROM (SELECT 1 AS x, 2 AS Y)) WHERE Y = '2'; == Parsed Logical Plan == 'Project [*] +- 'Filter ('Y = 2) +- 'Project ['x] +- Project [1 AS x#4, 2 AS Y#5] +- OneRowRelation$== Analyzed Logical Plan == x: int Project [x#4] +- Project [x#4] +- Filter (cast(Y#5 as bigint) = cast(2 as bigint)) +- Project [x#4, Y#5] +- Project [1 AS x#4, 2 AS Y#5] +- OneRowRelation$== Optimized Logical Plan == Project [1 AS x#4] +- OneRowRelation$== Physical Plan == *Project [1 AS x#4] +- Scan OneRowRelation[] Time taken: 0.813 seconds, Fetched 1 row(s) {code} - Do we care that this is a regression? That the query used to work in 2.1 but now breaks in later version? - Do we care that filter using non-existing columns as long as the column exists in the original table only work in certain case but not all cases (if you have SubqueryAlias) > Dropped columns still available for filtering > - > > Key: SPARK-30421 > URL: https://issues.apache.org/jira/browse/SPARK-30421 > Project: Spark > Issue Type: Bug > Components: Spark Core >Affects Versions: 2.4.4 >Reporter: Tobias Hermann >Priority: Minor > > The following minimal example: > {quote}val df = Seq((0, "a"), (1, "b")).toDF("foo", "bar") > df.select("foo").where($"bar" === "a").show > df.drop("bar").where($"bar" === "a").show > {quote} > should result in an error like the following: > {quote}org.apache.spark.sql.AnalysisException: cannot resolve '`bar`' given > input columns: [foo]; > {quote} > However, it does not but instead works without error, as if the column "bar" > would exist. -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-30421) Dropped columns still available for filtering
[ https://issues.apache.org/jira/browse/SPARK-30421?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17022900#comment-17022900 ] Tobias Hermann commented on SPARK-30421: [~dongjoon] I'm glad we are aligned now. :) For future reference: The original Pandas example {quote}df.drop(columns=["col1"]).loc[df["col1"] == 1] {quote} accesses the (unnamed) dataframe resulting from the drop call by row index (loc). This would even work (but not be very meaningful) by using a totally independent dataframe for this filtering. {quote}df_foo = pd.DataFrame(data=\{'foo': [0, 1]}) df_bar = pd.DataFrame(data=\{'bar': ["a", "b"]}) df_bar.loc[df_foo["foo"] == 1] {quote} > Dropped columns still available for filtering > - > > Key: SPARK-30421 > URL: https://issues.apache.org/jira/browse/SPARK-30421 > Project: Spark > Issue Type: Bug > Components: Spark Core >Affects Versions: 2.4.4 >Reporter: Tobias Hermann >Priority: Minor > > The following minimal example: > {quote}val df = Seq((0, "a"), (1, "b")).toDF("foo", "bar") > df.select("foo").where($"bar" === "a").show > df.drop("bar").where($"bar" === "a").show > {quote} > should result in an error like the following: > {quote}org.apache.spark.sql.AnalysisException: cannot resolve '`bar`' given > input columns: [foo]; > {quote} > However, it does not but instead works without error, as if the column "bar" > would exist. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-30421) Dropped columns still available for filtering
[ https://issues.apache.org/jira/browse/SPARK-30421?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17022850#comment-17022850 ] Dongjoon Hyun commented on SPARK-30421: --- While rethinking about this, the original column's index might be different because it can be considered a value array without any meaning. Got it, [~tobias_hermann]. > Dropped columns still available for filtering > - > > Key: SPARK-30421 > URL: https://issues.apache.org/jira/browse/SPARK-30421 > Project: Spark > Issue Type: Bug > Components: Spark Core >Affects Versions: 2.4.4 >Reporter: Tobias Hermann >Priority: Minor > > The following minimal example: > {quote}val df = Seq((0, "a"), (1, "b")).toDF("foo", "bar") > df.select("foo").where($"bar" === "a").show > df.drop("bar").where($"bar" === "a").show > {quote} > should result in an error like the following: > {quote}org.apache.spark.sql.AnalysisException: cannot resolve '`bar`' given > input columns: [foo]; > {quote} > However, it does not but instead works without error, as if the column "bar" > would exist. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-30421) Dropped columns still available for filtering
[ https://issues.apache.org/jira/browse/SPARK-30421?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17022835#comment-17022835 ] Dongjoon Hyun commented on SPARK-30421: --- Nope. Your example is different. I illustrated what I wanted. "Pandas supports filtering with *the original column's index* on the dropped data frame." That's my point. I intentionally didn't declare `df2` or `df2["bar"]`. > Dropped columns still available for filtering > - > > Key: SPARK-30421 > URL: https://issues.apache.org/jira/browse/SPARK-30421 > Project: Spark > Issue Type: Bug > Components: Spark Core >Affects Versions: 2.4.4 >Reporter: Tobias Hermann >Priority: Minor > > The following minimal example: > {quote}val df = Seq((0, "a"), (1, "b")).toDF("foo", "bar") > df.select("foo").where($"bar" === "a").show > df.drop("bar").where($"bar" === "a").show > {quote} > should result in an error like the following: > {quote}org.apache.spark.sql.AnalysisException: cannot resolve '`bar`' given > input columns: [foo]; > {quote} > However, it does not but instead works without error, as if the column "bar" > would exist. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-30421) Dropped columns still available for filtering
[ https://issues.apache.org/jira/browse/SPARK-30421?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17022719#comment-17022719 ] Tobias Hermann commented on SPARK-30421: [~dongjoon] No, that's different. To make it equivalent, you'd have to change your example to the following: {quote}import pandas as pd df = pd.DataFrame(data=\{'foo': [0, 1], 'bar': ["a", "b"]}) df2 = df.drop(columns=["bar"]) df2[df2["bar"] == "a"] {quote} And that correctly results in {quote}KeyError: 'bar' {quote} In Spark, however, the following code works without error: {quote}val df = Seq((0, "a"), (1, "b")).toDF("foo", "bar") val df2 = df.drop("bar") df2.where($"bar" === "a").show {quote} > Dropped columns still available for filtering > - > > Key: SPARK-30421 > URL: https://issues.apache.org/jira/browse/SPARK-30421 > Project: Spark > Issue Type: Bug > Components: Spark Core >Affects Versions: 2.4.4 >Reporter: Tobias Hermann >Priority: Minor > > The following minimal example: > {quote}val df = Seq((0, "a"), (1, "b")).toDF("foo", "bar") > df.select("foo").where($"bar" === "a").show > df.drop("bar").where($"bar" === "a").show > {quote} > should result in an error like the following: > {quote}org.apache.spark.sql.AnalysisException: cannot resolve '`bar`' given > input columns: [foo]; > {quote} > However, it does not but instead works without error, as if the column "bar" > would exist. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-30421) Dropped columns still available for filtering
[ https://issues.apache.org/jira/browse/SPARK-30421?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17022715#comment-17022715 ] Tobias Hermann commented on SPARK-30421: [~dongjoon] Thanks, I think that's not good. So I just opened a Pandas issue too. :D [https://github.com/pandas-dev/pandas/issues/31272] > Dropped columns still available for filtering > - > > Key: SPARK-30421 > URL: https://issues.apache.org/jira/browse/SPARK-30421 > Project: Spark > Issue Type: Bug > Components: Spark Core >Affects Versions: 2.4.4 >Reporter: Tobias Hermann >Priority: Minor > > The following minimal example: > {quote}val df = Seq((0, "a"), (1, "b")).toDF("foo", "bar") > df.select("foo").where($"bar" === "a").show > df.drop("bar").where($"bar" === "a").show > {quote} > should result in an error like the following: > {quote}org.apache.spark.sql.AnalysisException: cannot resolve '`bar`' given > input columns: [foo]; > {quote} > However, it does not but instead works without error, as if the column "bar" > would exist. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-30421) Dropped columns still available for filtering
[ https://issues.apache.org/jira/browse/SPARK-30421?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17022694#comment-17022694 ] Dongjoon Hyun commented on SPARK-30421: --- Technically, Python `pandas` follows the same lazy manner, [~tobias_hermann]. {code} >>> df col1 col2 0 1 3 1 2 4 >>> df.drop(columns=["col1"]).loc[df["col1"] == 1] col2 0 3 {code} > Dropped columns still available for filtering > - > > Key: SPARK-30421 > URL: https://issues.apache.org/jira/browse/SPARK-30421 > Project: Spark > Issue Type: Bug > Components: Spark Core >Affects Versions: 2.4.4 >Reporter: Tobias Hermann >Priority: Minor > > The following minimal example: > {quote}val df = Seq((0, "a"), (1, "b")).toDF("foo", "bar") > df.select("foo").where($"bar" === "a").show > df.drop("bar").where($"bar" === "a").show > {quote} > should result in an error like the following: > {quote}org.apache.spark.sql.AnalysisException: cannot resolve '`bar`' given > input columns: [foo]; > {quote} > However, it does not but instead works without error, as if the column "bar" > would exist. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-30421) Dropped columns still available for filtering
[ https://issues.apache.org/jira/browse/SPARK-30421?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17011941#comment-17011941 ] Tobias Hermann commented on SPARK-30421: It allows you to write code that should break but does not. Then, later, somebody does an innocent and totally valid refactoring, and suddenly the code is broken, and this poor person goes crazy. ;) > Dropped columns still available for filtering > - > > Key: SPARK-30421 > URL: https://issues.apache.org/jira/browse/SPARK-30421 > Project: Spark > Issue Type: Bug > Components: Spark Core >Affects Versions: 2.4.4 >Reporter: Tobias Hermann >Priority: Minor > > The following minimal example: > {quote}val df = Seq((0, "a"), (1, "b")).toDF("foo", "bar") > df.select("foo").where($"bar" === "a").show > df.drop("bar").where($"bar" === "a").show > {quote} > should result in an error like the following: > {quote}org.apache.spark.sql.AnalysisException: cannot resolve '`bar`' given > input columns: [foo]; > {quote} > However, it does not but instead works without error, as if the column "bar" > would exist. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-30421) Dropped columns still available for filtering
[ https://issues.apache.org/jira/browse/SPARK-30421?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17011930#comment-17011930 ] Wenchen Fan commented on SPARK-30421: - but it will not break anything, right? It just gives more chances to let your query compile. > Dropped columns still available for filtering > - > > Key: SPARK-30421 > URL: https://issues.apache.org/jira/browse/SPARK-30421 > Project: Spark > Issue Type: Bug > Components: Spark Core >Affects Versions: 2.4.4 >Reporter: Tobias Hermann >Priority: Minor > > The following minimal example: > {quote}val df = Seq((0, "a"), (1, "b")).toDF("foo", "bar") > df.select("foo").where($"bar" === "a").show > df.drop("bar").where($"bar" === "a").show > {quote} > should result in an error like the following: > {quote}org.apache.spark.sql.AnalysisException: cannot resolve '`bar`' given > input columns: [foo]; > {quote} > However, it does not but instead works without error, as if the column "bar" > would exist. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-30421) Dropped columns still available for filtering
[ https://issues.apache.org/jira/browse/SPARK-30421?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17011422#comment-17011422 ] Tobias Hermann commented on SPARK-30421: [~cloud_fan] I think it is an issue because it means one can not simply look at the schema of a dataframe to determine if an operation is valid. Instead one has to consider the whole history of how the dataframe was created/derived. This leads to the effect that refactorings, e.g., changing the way of creation of a dataframe, will break one's code, even though the refactoring should be totally OK because it results in the exact same dataframe schema. > Dropped columns still available for filtering > - > > Key: SPARK-30421 > URL: https://issues.apache.org/jira/browse/SPARK-30421 > Project: Spark > Issue Type: Bug > Components: Spark Core >Affects Versions: 2.4.4 >Reporter: Tobias Hermann >Priority: Minor > > The following minimal example: > {quote}val df = Seq((0, "a"), (1, "b")).toDF("foo", "bar") > df.select("foo").where($"bar" === "a").show > df.drop("bar").where($"bar" === "a").show > {quote} > should result in an error like the following: > {quote}org.apache.spark.sql.AnalysisException: cannot resolve '`bar`' given > input columns: [foo]; > {quote} > However, it does not but instead works without error, as if the column "bar" > would exist. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-30421) Dropped columns still available for filtering
[ https://issues.apache.org/jira/browse/SPARK-30421?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17011390#comment-17011390 ] Wenchen Fan commented on SPARK-30421: - is this really an issue? The current semantic seems like: you can filter using non-existing columns as long as the column exists in the original table. > Dropped columns still available for filtering > - > > Key: SPARK-30421 > URL: https://issues.apache.org/jira/browse/SPARK-30421 > Project: Spark > Issue Type: Bug > Components: Spark Core >Affects Versions: 2.4.4 >Reporter: Tobias Hermann >Priority: Minor > > The following minimal example: > {quote}val df = Seq((0, "a"), (1, "b")).toDF("foo", "bar") > df.select("foo").where($"bar" === "a").show > df.drop("bar").where($"bar" === "a").show > {quote} > should result in an error like the following: > {quote}org.apache.spark.sql.AnalysisException: cannot resolve '`bar`' given > input columns: [foo]; > {quote} > However, it does not but instead works without error, as if the column "bar" > would exist. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-30421) Dropped columns still available for filtering
[ https://issues.apache.org/jira/browse/SPARK-30421?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17010271#comment-17010271 ] Takeshi Yamamuro commented on SPARK-30421: -- Based on the current implementation, drop and select (drop is a shorthand for a partial use-case of select?) seems to have the same semantics. If so, that query might be correct in lazy evaluation. btw, for changing this behaviour, IMO it would be better to reconstruct dataframe([https://github.com/maropu/spark/commit/fac04161405b9ee755b4c7f87de2a144c609c7fa]) instead of modifying the resolution logic. That's because the resolution logic affects many places. > Dropped columns still available for filtering > - > > Key: SPARK-30421 > URL: https://issues.apache.org/jira/browse/SPARK-30421 > Project: Spark > Issue Type: Bug > Components: Spark Core >Affects Versions: 2.4.4 >Reporter: Tobias Hermann >Priority: Minor > > The following minimal example: > {quote}val df = Seq((0, "a"), (1, "b")).toDF("foo", "bar") > df.select("foo").where($"bar" === "a").show > df.drop("bar").where($"bar" === "a").show > {quote} > should result in an error like the following: > {quote}org.apache.spark.sql.AnalysisException: cannot resolve '`bar`' given > input columns: [foo]; > {quote} > However, it does not but instead works without error, as if the column "bar" > would exist. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-30421) Dropped columns still available for filtering
[ https://issues.apache.org/jira/browse/SPARK-30421?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17010227#comment-17010227 ] Aman Omer commented on SPARK-30421: --- cc [~maropu] > Dropped columns still available for filtering > - > > Key: SPARK-30421 > URL: https://issues.apache.org/jira/browse/SPARK-30421 > Project: Spark > Issue Type: Bug > Components: Spark Core >Affects Versions: 2.4.4 >Reporter: Tobias Hermann >Priority: Minor > > The following minimal example: > {quote}val df = Seq((0, "a"), (1, "b")).toDF("foo", "bar") > df.select("foo").where($"bar" === "a").show > df.drop("bar").where($"bar" === "a").show > {quote} > should result in an error like the following: > {quote}org.apache.spark.sql.AnalysisException: cannot resolve '`bar`' given > input columns: [foo]; > {quote} > However, it does not but instead works without error, as if the column "bar" > would exist. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-30421) Dropped columns still available for filtering
[ https://issues.apache.org/jira/browse/SPARK-30421?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17008708#comment-17008708 ] Aman Omer commented on SPARK-30421: --- I am panning to add a marker to Project when it is introduced via Drop command which can escape ResolveMissingReferences rule. [~cloud_fan] [~srowen] Kindly provide feedback about this. > Dropped columns still available for filtering > - > > Key: SPARK-30421 > URL: https://issues.apache.org/jira/browse/SPARK-30421 > Project: Spark > Issue Type: Bug > Components: Spark Core >Affects Versions: 2.4.4 >Reporter: Tobias Hermann >Priority: Minor > > The following minimal example: > {quote}val df = Seq((0, "a"), (1, "b")).toDF("foo", "bar") > df.select("foo").where($"bar" === "a").show > df.drop("bar").where($"bar" === "a").show > {quote} > should result in an error like the following: > {quote}org.apache.spark.sql.AnalysisException: cannot resolve '`bar`' given > input columns: [foo]; > {quote} > However, it does not but instead works without error, as if the column "bar" > would exist. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-30421) Dropped columns still available for filtering
[ https://issues.apache.org/jira/browse/SPARK-30421?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17008705#comment-17008705 ] Aman Omer commented on SPARK-30421: --- In Analysis planning phase, rule _*ResolveMissingReferences*_ is adding +foo+ in Filter's child. I think that is why in this case, it is not showing any exception. Here we should be able to distinguish between cases when we want to add missing columns and when we don't (like in case of drop). > Dropped columns still available for filtering > - > > Key: SPARK-30421 > URL: https://issues.apache.org/jira/browse/SPARK-30421 > Project: Spark > Issue Type: Bug > Components: Spark Core >Affects Versions: 2.4.4 >Reporter: Tobias Hermann >Priority: Minor > > The following minimal example: > {quote}val df = Seq((0, "a"), (1, "b")).toDF("foo", "bar") > df.select("foo").where($"bar" === "a").show > df.drop("bar").where($"bar" === "a").show > {quote} > should result in an error like the following: > {quote}org.apache.spark.sql.AnalysisException: cannot resolve '`bar`' given > input columns: [foo]; > {quote} > However, it does not but instead works without error, as if the column "bar" > would exist. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-30421) Dropped columns still available for filtering
[ https://issues.apache.org/jira/browse/SPARK-30421?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17008699#comment-17008699 ] Aman Omer commented on SPARK-30421: --- Plans for {code:java} df.drop("bar").where($"bar" === "a").explain(true) {code} {code:java} == Parsed Logical Plan == 'Filter ('bar = a) +- Project [foo#7] +- Project [_1#2 AS foo#7, _2#3 AS bar#8] +- LocalRelation [_1#2, _2#3] == Analyzed Logical Plan == foo: int Project [foo#7] +- Filter (bar#8 = a) +- Project [foo#7, bar#8] +- Project [_1#2 AS foo#7, _2#3 AS bar#8] +- LocalRelation [_1#2, _2#3] == Optimized Logical Plan == LocalRelation [foo#7] == Physical Plan == LocalTableScan [foo#7] {code} > Dropped columns still available for filtering > - > > Key: SPARK-30421 > URL: https://issues.apache.org/jira/browse/SPARK-30421 > Project: Spark > Issue Type: Bug > Components: Spark Core >Affects Versions: 2.4.4 >Reporter: Tobias Hermann >Priority: Minor > > The following minimal example: > {quote}val df = Seq((0, "a"), (1, "b")).toDF("foo", "bar") > df.select("foo").where($"bar" === "a").show > df.drop("bar").where($"bar" === "a").show > {quote} > should result in an error like the following: > {quote}org.apache.spark.sql.AnalysisException: cannot resolve '`bar`' given > input columns: [foo]; > {quote} > However, it does not but instead works without error, as if the column "bar" > would exist. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-30421) Dropped columns still available for filtering
[ https://issues.apache.org/jira/browse/SPARK-30421?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17008640#comment-17008640 ] Aman Omer commented on SPARK-30421: --- Thanks [~tobias_hermann] for reporting this issue. It seems, while resolving Filter in analysis phase, its child (that is, Project) is getting modified. I will investigate this further and update here. > Dropped columns still available for filtering > - > > Key: SPARK-30421 > URL: https://issues.apache.org/jira/browse/SPARK-30421 > Project: Spark > Issue Type: Bug > Components: Spark Core >Affects Versions: 2.4.4 >Reporter: Tobias Hermann >Priority: Minor > > The following minimal example: > {quote}val df = Seq((0, "a"), (1, "b")).toDF("foo", "bar") > df.select("foo").where($"bar" === "a").show > df.drop("bar").where($"bar" === "a").show > {quote} > should result in an error like the following: > {quote}org.apache.spark.sql.AnalysisException: cannot resolve '`bar`' given > input columns: [foo]; > {quote} > However, it does not but instead works without error, as if the column "bar" > would exist. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-30421) Dropped columns still available for filtering
[ https://issues.apache.org/jira/browse/SPARK-30421?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17008301#comment-17008301 ] Tobias Hermann commented on SPARK-30421: see: [https://stackoverflow.com/questions/59597678/why-does-filtering-on-a-non-existing-non-selected-column-work] > Dropped columns still available for filtering > - > > Key: SPARK-30421 > URL: https://issues.apache.org/jira/browse/SPARK-30421 > Project: Spark > Issue Type: Bug > Components: Spark Core >Affects Versions: 2.4.4 >Reporter: Tobias Hermann >Priority: Minor > > The following minimal example: > {quote}val df = Seq((0, "a"), (1, "b")).toDF("foo", "bar") > df.select("foo").where($"bar" === "a").show > df.drop("bar").where($"bar" === "a").show > {quote} > should result in an error like the following: > {quote}org.apache.spark.sql.AnalysisException: cannot resolve '`bar`' given > input columns: [foo]; > {quote} > However, it does not but instead works without error, as if the column "bar" > would exist. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org