-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-46677][SQL][CONNECT] Fix dataframe["*"] resolution
#44689
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change | ||||||||||||||||||||||||||||||||||||||
|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|
|
|
@@ -76,15 +76,6 @@ | |||||||||||||||||||||||||||||||||||||||
| from pyspark.sql.connect.udtf import UserDefinedTableFunction | ||||||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||||||
| def _to_col_with_plan_id(col: str, plan_id: Optional[int]) -> Column: | ||||||||||||||||||||||||||||||||||||||||
|
||||||||||||||||||||||||||||||||||||||||
| def col(colName: String): Column = colName match { | |
| case "*" => | |
| Column(ResolvedStar(queryExecution.analyzed.output)) | |
| case _ => | |
| if (sparkSession.sessionState.conf.supportQuotedRegexColumnName) { | |
| colRegex(colName) | |
| } else { | |
| Column(addDataFrameIdToCol(resolve(colName))) | |
| } | |
| } |
spark/sql/core/src/main/scala/org/apache/spark/sql/Column.scala
Lines 154 to 162 in 0a79199
| def this(name: String) = this(withOrigin { | |
| name match { | |
| case "*" => UnresolvedStar(None) | |
| case _ if name.endsWith(".*") => | |
| val parts = UnresolvedAttribute.parseAttributeName(name.substring(0, name.length - 2)) | |
| UnresolvedStar(Some(parts)) | |
| case _ => UnresolvedAttribute.quotedString(name) | |
| } | |
| }) |
Large diffs are not rendered by default.
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -558,6 +558,44 @@ def test_invalid_column(self): | |
| ): | ||
| cdf1.select(cdf2.a).schema | ||
|
|
||
| def test_invalid_star(self): | ||
|
||
| data1 = [Row(a=1, b=2, c=3)] | ||
| cdf1 = self.connect.createDataFrame(data1) | ||
|
|
||
| data2 = [Row(a=2, b=0)] | ||
| cdf2 = self.connect.createDataFrame(data2) | ||
|
|
||
| # Can find the target plan node, but fail to resolve with it | ||
| with self.assertRaisesRegex( | ||
| AnalysisException, | ||
| "CANNOT_RESOLVE_DATAFRAME_COLUMN", | ||
| ): | ||
| cdf3 = cdf1.select(cdf1.a) | ||
| cdf3.select(cdf1["*"]).schema | ||
|
|
||
| # Can find the target plan node, but fail to resolve with it | ||
| with self.assertRaisesRegex( | ||
| AnalysisException, | ||
| "CANNOT_RESOLVE_DATAFRAME_COLUMN", | ||
| ): | ||
| # column 'a has been replaced | ||
| cdf3 = cdf1.withColumn("a", CF.lit(0)) | ||
| cdf3.select(cdf1["*"]).schema | ||
|
|
||
| # Can not find the target plan node by plan id | ||
| with self.assertRaisesRegex( | ||
| AnalysisException, | ||
| "CANNOT_RESOLVE_DATAFRAME_COLUMN", | ||
| ): | ||
| cdf1.select(cdf2["*"]).schema | ||
|
|
||
| # cdf1["*"] exists on both side | ||
| with self.assertRaisesRegex( | ||
| AnalysisException, | ||
| "AMBIGUOUS_COLUMN_REFERENCE", | ||
| ): | ||
| cdf1.join(cdf1).select(cdf1["*"]).schema | ||
|
|
||
| def test_collect(self): | ||
| cdf = self.connect.read.table(self.tbl_name) | ||
| sdf = self.spark.read.table(self.tbl_name) | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -69,6 +69,41 @@ def test_range(self): | |
| self.assertEqual(self.spark.range(-2).count(), 0) | ||
| self.assertEqual(self.spark.range(3).count(), 3) | ||
|
|
||
| def test_dataframe_star(self): | ||
|
||
| df1 = self.spark.createDataFrame([{"a": 1}]) | ||
| df2 = self.spark.createDataFrame([{"a": 1, "b": "v"}]) | ||
| df3 = df2.withColumnsRenamed({"a": "x", "b": "y"}) | ||
|
|
||
| df = df1.join(df2) | ||
| self.assertEqual(df.columns, ["a", "a", "b"]) | ||
| self.assertEqual(df.select(df1["*"]).columns, ["a"]) | ||
| self.assertEqual(df.select(df2["*"]).columns, ["a", "b"]) | ||
|
|
||
| df = df1.join(df2).withColumn("c", lit(0)) | ||
| self.assertEqual(df.columns, ["a", "a", "b", "c"]) | ||
| self.assertEqual(df.select(df1["*"]).columns, ["a"]) | ||
| self.assertEqual(df.select(df2["*"]).columns, ["a", "b"]) | ||
|
|
||
| df = df1.join(df2, "a") | ||
| self.assertEqual(df.columns, ["a", "b"]) | ||
| self.assertEqual(df.select(df1["*"]).columns, ["a"]) | ||
| self.assertEqual(df.select(df2["*"]).columns, ["a", "b"]) | ||
|
|
||
| df = df1.join(df2, "a").withColumn("c", lit(0)) | ||
| self.assertEqual(df.columns, ["a", "b", "c"]) | ||
| self.assertEqual(df.select(df1["*"]).columns, ["a"]) | ||
| self.assertEqual(df.select(df2["*"]).columns, ["a", "b"]) | ||
|
|
||
| df = df2.join(df3) | ||
| self.assertEqual(df.columns, ["a", "b", "x", "y"]) | ||
| self.assertEqual(df.select(df2["*"]).columns, ["a", "b"]) | ||
| self.assertEqual(df.select(df3["*"]).columns, ["x", "y"]) | ||
|
|
||
| df = df2.join(df3).withColumn("c", lit(0)) | ||
| self.assertEqual(df.columns, ["a", "b", "x", "y", "c"]) | ||
| self.assertEqual(df.select(df2["*"]).columns, ["a", "b"]) | ||
| self.assertEqual(df.select(df3["*"]).columns, ["x", "y"]) | ||
|
|
||
| def test_self_join(self): | ||
| df1 = self.spark.range(10).withColumn("a", lit(0)) | ||
| df2 = df1.withColumnRenamed("a", "b") | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
TODO for myself, should revisit the implementation of
colRegexThere was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We can probably skip it in spark connect. It's really a weird feature and non-standard.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It's off by default anyway, so we can throw a proper error if it's enabled in spark connect.