This is an automated email from the ASF dual-hosted git repository. gurwls223 pushed a commit to branch branch-3.4 in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/branch-3.4 by this push: new c176000a0c0 [SPARK-41999][CONNECT][PYTHON] Fix bucketBy/sortBy to properly use the first column name c176000a0c0 is described below commit c176000a0c0d2d50a4f11ae227d1f9e5a494319c Author: Takuya UESHIN <ues...@databricks.com> AuthorDate: Tue Feb 14 12:55:16 2023 +0900 [SPARK-41999][CONNECT][PYTHON] Fix bucketBy/sortBy to properly use the first column name ### What changes were proposed in this pull request? Fixes `DataFrameWriter.bucketBy` and `sortBy` to porperly use the first column name. ### Why are the changes needed? Currently `DataFrameWriter.bucketBy` and `sortBy` mistakenly drop the first column name, which ends with `NoSuchElementException`. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Enabled related tests. Closes #40002 from ueshin/issues/SPARK-41999/bucketBy. Authored-by: Takuya UESHIN <ues...@databricks.com> Signed-off-by: Hyukjin Kwon <gurwls...@apache.org> (cherry picked from commit 18aa6e381bb6440ee5bc655edfd9aeddcedd30f8) Signed-off-by: Hyukjin Kwon <gurwls...@apache.org> --- python/pyspark/sql/connect/readwriter.py | 6 ++---- python/pyspark/sql/tests/connect/test_parity_readwriter.py | 5 ----- 2 files changed, 2 insertions(+), 9 deletions(-) diff --git a/python/pyspark/sql/connect/readwriter.py b/python/pyspark/sql/connect/readwriter.py index ee4e3018bb9..5d886a694cf 100644 --- a/python/pyspark/sql/connect/readwriter.py +++ b/python/pyspark/sql/connect/readwriter.py @@ -421,7 +421,7 @@ class DataFrameWriter(OptionUtils): raise TypeError("all names should be `str`") self._write.num_buckets = numBuckets - self._write.bucket_cols = cast(List[str], cols) + self._write.bucket_cols = cast(List[str], [col, *cols]) return self bucketBy.__doc__ = PySparkDataFrameWriter.bucketBy.__doc__ @@ -446,7 +446,7 @@ class DataFrameWriter(OptionUtils): if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)): raise TypeError("all names should be `str`") - self._write.sort_cols = cast(List[str], cols) + self._write.sort_cols = cast(List[str], [col, *cols]) return self sortBy.__doc__ = PySparkDataFrameWriter.sortBy.__doc__ @@ -694,8 +694,6 @@ def _test() -> None: # TODO(SPARK-41817): Support reading with schema del pyspark.sql.connect.readwriter.DataFrameReader.option.__doc__ del pyspark.sql.connect.readwriter.DataFrameWriter.option.__doc__ - del pyspark.sql.connect.readwriter.DataFrameWriter.bucketBy.__doc__ - del pyspark.sql.connect.readwriter.DataFrameWriter.sortBy.__doc__ # TODO(SPARK-42426): insertInto fails when the column names are different from the table columns del pyspark.sql.connect.readwriter.DataFrameWriter.insertInto.__doc__ diff --git a/python/pyspark/sql/tests/connect/test_parity_readwriter.py b/python/pyspark/sql/tests/connect/test_parity_readwriter.py index db713c3bee5..61f542385ec 100644 --- a/python/pyspark/sql/tests/connect/test_parity_readwriter.py +++ b/python/pyspark/sql/tests/connect/test_parity_readwriter.py @@ -21,11 +21,6 @@ from pyspark.testing.connectutils import ReusedConnectTestCase class ReadwriterParityTests(ReadwriterTestsMixin, ReusedConnectTestCase): - # TODO(SPARK-41999): NPE for bucketed write - @unittest.skip("Fails in Spark Connect, should enable.") - def test_bucketed_write(self): - super().test_bucketed_write() - # TODO(SPARK-41834): Implement SparkSession.conf @unittest.skip("Fails in Spark Connect, should enable.") def test_save_and_load(self): --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org