(spark) branch master updated: [SPARK-49995][SQL] Add named argument support to more TVFs
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new b81da764b9c5 [SPARK-49995][SQL] Add named argument support to more TVFs b81da764b9c5 is described below commit b81da764b9c562efeb820e8d3850a8e7a60c7bf7 Author: Takuya Ueshin AuthorDate: Tue Nov 5 12:45:28 2024 -0800 [SPARK-49995][SQL] Add named argument support to more TVFs ### What changes were proposed in this pull request? Add named argument support to more TVFs. ```sql SELECT * FROM inline(input => array(struct(1, 'a'), struct(2, 'b'))); SELECT * FROM inline_outer(input => array(struct(1, 'a'), struct(2, 'b'))); SELECT * FROM posexplode(collection => array(1, 2)); SELECT * FROM posexplode_outer(collection => map('a', 1, 'b', 2)); SELECT * FROM variant_explode(input => parse_json('["hello", "world"]')); SELECT * FROM variant_explode_outer(input => parse_json('{"a": true, "b": 3.14}')); ``` ### Why are the changes needed? The following TVFs should support named argument as same as `explode`: - `inline` - `posexplode` - `variant_explode` and their `_outer` variations. ### Does this PR introduce _any_ user-facing change? Yes, the above functions support named arguments. ### How was this patch tested? Added the related tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48503 from ueshin/issues/SPARK-49995/named_argument. Authored-by: Takuya Ueshin Signed-off-by: Takuya Ueshin --- .../sql/catalyst/analysis/FunctionRegistry.scala | 20 +-- .../sql/catalyst/expressions/generators.scala | 156 + .../expressions/variant/variantExpressions.scala | 66 ++--- .../named-function-arguments.sql.out | 98 + .../variant/named-function-arguments.sql.out | 48 +++ .../sql-tests/inputs/named-function-arguments.sql | 14 ++ .../inputs/variant/named-function-arguments.sql| 6 + .../results/named-function-arguments.sql.out | 96 + .../variant/named-function-arguments.sql.out | 48 +++ .../thriftserver/ThriftServerQueryTestSuite.scala | 4 +- 10 files changed, 503 insertions(+), 53 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 4ad0b81b8f26..8fe5c1b59902 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -367,8 +367,8 @@ object FunctionRegistry { expressionGeneratorBuilderOuter("explode_outer", ExplodeExpressionBuilder), expression[Greatest]("greatest"), expression[If]("if"), -expression[Inline]("inline"), -expressionGeneratorOuter[Inline]("inline_outer"), +expressionBuilder("inline", InlineExpressionBuilder), +expressionGeneratorBuilderOuter("inline_outer", InlineExpressionBuilder), expression[IsNaN]("isnan"), expression[Nvl]("ifnull", setAlias = true), expression[IsNull]("isnull"), @@ -379,8 +379,8 @@ object FunctionRegistry { expression[NullIfZero]("nullifzero"), expression[Nvl]("nvl"), expression[Nvl2]("nvl2"), -expression[PosExplode]("posexplode"), -expressionGeneratorOuter[PosExplode]("posexplode_outer"), +expressionBuilder("posexplode", PosExplodeExpressionBuilder), +expressionGeneratorBuilderOuter("posexplode_outer", PosExplodeExpressionBuilder), expression[Rand]("rand"), expression[Rand]("random", true, Some("3.0.0")), expression[Randn]("randn"), @@ -1172,16 +1172,16 @@ object TableFunctionRegistry { logicalPlan[Range]("range"), generatorBuilder("explode", ExplodeGeneratorBuilder), generatorBuilder("explode_outer", ExplodeOuterGeneratorBuilder), -generator[Inline]("inline"), -generator[Inline]("inline_outer", outer = true), +generatorBuilder("inline", InlineGeneratorBuilder), +generatorBuilder("inline_outer", InlineOuterGeneratorBuilder), generator[JsonTuple]("json_tuple"), -generator[PosExplode]("posexplode"), -generator[PosExplode]
(spark) branch master updated: [SPARK-49009][SQL][PYTHON] Make Column APIs and functions accept Enums
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 7e678a07ecd1 [SPARK-49009][SQL][PYTHON] Make Column APIs and functions accept Enums 7e678a07ecd1 is described below commit 7e678a07ecd144eae251e5ca718892a19f8aef72 Author: Takuya Ueshin AuthorDate: Fri Jul 26 11:49:27 2024 -0700 [SPARK-49009][SQL][PYTHON] Make Column APIs and functions accept Enums ### What changes were proposed in this pull request? Make Column APIs and functions accept `Enum`s. ### Why are the changes needed? `Enum`s can be accepted in Column APIs and functions using its `value`. ```py >>> from pyspark.sql import functions as F >>> from enum import Enum >>> class A(Enum): ... X = "x" ... Y = "y" ... >>> F.lit(A.X) Column<'x'> >>> F.lit(A.X) + A.Y Column<'`+`(x, y)'> ``` ### Does this PR introduce _any_ user-facing change? Yes, Python's `Enum`s will be used as literal values. ### How was this patch tested? Added the related tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47495 from ueshin/issues/SPARK-49009/enum. Authored-by: Takuya Ueshin Signed-off-by: Takuya Ueshin --- python/pyspark/sql/classic/column.py| 19 ++- python/pyspark/sql/connect/column.py| 6 + python/pyspark/sql/connect/expressions.py | 4 +- python/pyspark/sql/connect/functions/builtin.py | 52 ++- python/pyspark/sql/functions/builtin.py | 188 python/pyspark/sql/tests/test_column.py | 65 python/pyspark/sql/tests/test_functions.py | 30 python/pyspark/sql/utils.py | 6 + 8 files changed, 295 insertions(+), 75 deletions(-) diff --git a/python/pyspark/sql/classic/column.py b/python/pyspark/sql/classic/column.py index 92c57438c739..2504ec740648 100644 --- a/python/pyspark/sql/classic/column.py +++ b/python/pyspark/sql/classic/column.py @@ -35,7 +35,7 @@ from pyspark.sql.column import Column as ParentColumn from pyspark.errors import PySparkAttributeError, PySparkTypeError, PySparkValueError from pyspark.errors.utils import with_origin_to_class from pyspark.sql.types import DataType -from pyspark.sql.utils import get_active_spark_context +from pyspark.sql.utils import get_active_spark_context, enum_to_value if TYPE_CHECKING: from py4j.java_gateway import JavaObject @@ -52,7 +52,7 @@ def _create_column_from_literal( from py4j.java_gateway import JVMView sc = get_active_spark_context() -return cast(JVMView, sc._jvm).functions.lit(literal) +return cast(JVMView, sc._jvm).functions.lit(enum_to_value(literal)) def _create_column_from_name(name: str) -> "JavaObject": @@ -163,7 +163,7 @@ def _bin_op( other: Union[ParentColumn, "LiteralType", "DecimalLiteral", "DateTimeLiteral"], ) -> ParentColumn: """Create a method for given binary operator""" -jc = other._jc if isinstance(other, ParentColumn) else other +jc = other._jc if isinstance(other, ParentColumn) else enum_to_value(other) njc = getattr(self._jc, name)(jc) return Column(njc) @@ -441,20 +441,23 @@ class Column(ParentColumn): return _bin_op("endsWith", self, other) def like(self: ParentColumn, other: str) -> ParentColumn: -njc = getattr(self._jc, "like")(other) +njc = getattr(self._jc, "like")(enum_to_value(other)) return Column(njc) def rlike(self: ParentColumn, other: str) -> ParentColumn: -njc = getattr(self._jc, "rlike")(other) +njc = getattr(self._jc, "rlike")(enum_to_value(other)) return Column(njc) def ilike(self: ParentColumn, other: str) -> ParentColumn: -njc = getattr(self._jc, "ilike")(other) +njc = getattr(self._jc, "ilike")(enum_to_value(other)) return Column(njc) def substr( self, startPos: Union[int, ParentColumn], length: Union[int, ParentColumn] ) -> ParentColumn: +startPos = enum_to_value(startPos) +length = enum_to_value(length) + if type(startPos) != type(length): raise PySparkTypeError( errorClass="NOT_SAME_TYPE", @@ -586,12 +589,12 @@ class Column(ParentColumn): errorClass="NOT_COLUMN", messageParameters={"arg_name": "condition", "arg_type": type(condition).__name__}, ) -
(spark) branch master updated: [SPARK-48996][SQL][PYTHON] Allow bare literals for __and__ and __or__ of Column
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 78b83fa542a5 [SPARK-48996][SQL][PYTHON] Allow bare literals for __and__ and __or__ of Column 78b83fa542a5 is described below commit 78b83fa542a513efbfadf8e03358a4411175bbf3 Author: Takuya Ueshin AuthorDate: Thu Jul 25 13:47:01 2024 -0700 [SPARK-48996][SQL][PYTHON] Allow bare literals for __and__ and __or__ of Column ### What changes were proposed in this pull request? Allows bare literals for `__and__` and `__or__` of Column API in Spark Classic. ### Why are the changes needed? Currently bare literals are not allowed for `__and__` and `__or__` of Column API in Spark Classic and need to wrap with `lit()` function. It should be allowed similar to other similar operators. ```py >>> from pyspark.sql.functions import * >>> c = col("c") >>> c & True Traceback (most recent call last): ... py4j.Py4JException: Method and([class java.lang.Boolean]) does not exist >>> c & lit(True) Column<'and(c, true)'> ``` whereas other operators: ```py >>> c + 1 Column<'`+`(c, 1)'> >>> c + lit(1) Column<'`+`(c, 1)'> ``` Spark Connect allows this. ```py >>> c & True Column<'and(c, True)'> >>> c & lit(True) Column<'and(c, True)'> ``` ### Does this PR introduce _any_ user-facing change? Yes. ### How was this patch tested? Added the related tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47474 from ueshin/issues/SPARK-48996/literal_and_or. Authored-by: Takuya Ueshin Signed-off-by: Takuya Ueshin --- python/pyspark/sql/classic/column.py| 16 python/pyspark/sql/tests/test_column.py | 8 2 files changed, 20 insertions(+), 4 deletions(-) diff --git a/python/pyspark/sql/classic/column.py b/python/pyspark/sql/classic/column.py index c893050872f4..92c57438c739 100644 --- a/python/pyspark/sql/classic/column.py +++ b/python/pyspark/sql/classic/column.py @@ -309,12 +309,16 @@ class Column(ParentColumn): def __and__( self, other: Union[ParentColumn, "LiteralType", "DecimalLiteral", "DateTimeLiteral"] ) -> ParentColumn: -return _bin_op("and", self, other) +from pyspark.sql.functions import lit + +return _bin_op("and", self, lit(other)) def __or__( self, other: Union[ParentColumn, "LiteralType", "DecimalLiteral", "DateTimeLiteral"] ) -> ParentColumn: -return _bin_op("or", self, other) +from pyspark.sql.functions import lit + +return _bin_op("or", self, lit(other)) def __invert__(self) -> ParentColumn: return _func_op("not", self) @@ -322,12 +326,16 @@ class Column(ParentColumn): def __rand__( self, other: Union[ParentColumn, "LiteralType", "DecimalLiteral", "DateTimeLiteral"] ) -> ParentColumn: -return _bin_op("and", self, other) +from pyspark.sql.functions import lit + +return _bin_op("and", self, lit(other)) def __ror__( self, other: Union[ParentColumn, "LiteralType", "DecimalLiteral", "DateTimeLiteral"] ) -> ParentColumn: -return _bin_op("or", self, other) +from pyspark.sql.functions import lit + +return _bin_op("or", self, lit(other)) # container operators def __contains__(self, item: Any) -> None: diff --git a/python/pyspark/sql/tests/test_column.py b/python/pyspark/sql/tests/test_column.py index ac599fab1357..ecfcae36c955 100644 --- a/python/pyspark/sql/tests/test_column.py +++ b/python/pyspark/sql/tests/test_column.py @@ -94,6 +94,14 @@ class ColumnTestsMixin: cs.startswith("a"), cs.endswith("a"), ci.eqNullSafe(cs), +sf.col("b") & sf.lit(True), +sf.col("b") & True, +sf.lit(True) & sf.col("b"), +True & sf.col("b"), +sf.col("b") | sf.lit(True), +sf.col("b") | True, +sf.lit(True) | sf.col("b"), +True | sf.col("b"), ) self.assertTrue(all(isinstance(c, Column) for c in css)) self.assertTrue(isinstance(ci.cast(LongType()), Column)) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated (c4085f1f6f58 -> b062d4436f2b)
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from c4085f1f6f58 [SPARK-48776] Fix timestamp formatting for json, xml and csv add b062d4436f2b [SPARK-48798][PYTHON] Introduce `spark.profile.render` for SparkSession-based profiling No new revisions were added by this update. Summary of changes: dev/requirements.txt | 1 + docs/img/pyspark-udf-profile.png | Bin 0 -> 281384 bytes python/docs/source/development/debugging.rst | 29 + python/docs/source/getting_started/install.rst | 4 + python/mypy.ini| 3 + python/pyspark/sql/profiler.py | 117 - .../sql/tests/connect/test_parity_udf_profiler.py | 9 +- python/pyspark/sql/tests/test_udf_profiler.py | 104 ++ python/pyspark/worker.py | 4 +- 9 files changed, 267 insertions(+), 4 deletions(-) create mode 100644 docs/img/pyspark-udf-profile.png - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated (66d8a29dd7b4 -> 0864bbeac399)
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 66d8a29dd7b4 [SPARK-47577][SPARK-47579] Correct misleading usage of log key TASK_ID add 0864bbeac399 [SPARK-48566][PYTHON] Fix bug where partition indices are incorrect when UDTF analyze() uses both select and partitionColumns No new revisions were added by this update. Summary of changes: .../spark/sql/catalyst/analysis/Analyzer.scala | 14 +-- .../FunctionTableSubqueryArgumentExpression.scala | 7 +++- .../sql-tests/analyzer-results/udtf/udtf.sql.out | 20 ++ .../test/resources/sql-tests/inputs/udtf/udtf.sql | 16 .../resources/sql-tests/results/udtf/udtf.sql.out | 26 + .../apache/spark/sql/IntegratedUDFTestUtils.scala | 44 ++ 6 files changed, 122 insertions(+), 5 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch branch-3.5 updated: [SPARK-47921][CONNECT] Fix ExecuteJobTag creation in ExecuteHolder
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new fdc0cee17908 [SPARK-47921][CONNECT] Fix ExecuteJobTag creation in ExecuteHolder fdc0cee17908 is described below commit fdc0cee179087c31eff9aa1471870ee8a6228b1d Author: allisonwang-db AuthorDate: Wed Apr 24 18:22:29 2024 -0700 [SPARK-47921][CONNECT] Fix ExecuteJobTag creation in ExecuteHolder ### What changes were proposed in this pull request? This PR fixes a bug in the ExecuteJobTag creation in ExecuteHolder. The sessionId and userId are reversed. https://github.com/apache/spark/blob/8aa8ad6be7b3eeceafa2ad1e9211fb8133bb675c/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteHolder.scala#L296-L299 ### Why are the changes needed? To fix a bug ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing test ### Was this patch authored or co-authored using generative AI tooling? No Closes #46140 from allisonwang-db/spark-47921-execute-job-tag. Authored-by: allisonwang-db Signed-off-by: Takuya UESHIN (cherry picked from commit 5a1559a7ef03641db223a8b0862e3dd53032ead0) Signed-off-by: Takuya UESHIN --- .../scala/org/apache/spark/sql/connect/service/ExecuteHolder.scala | 2 +- .../spark/sql/connect/ui/SparkConnectServerListenerSuite.scala | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteHolder.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteHolder.scala index 5cf63c2195ab..0e4f344da901 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteHolder.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteHolder.scala @@ -285,7 +285,7 @@ private[connect] class ExecuteHolder( object ExecuteJobTag { private val prefix = "SparkConnect_OperationTag" - def apply(sessionId: String, userId: String, operationId: String): String = { + def apply(userId: String, sessionId: String, operationId: String): String = { s"${prefix}_" + s"User_${userId}_" + s"Session_${sessionId}_" + diff --git a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/ui/SparkConnectServerListenerSuite.scala b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/ui/SparkConnectServerListenerSuite.scala index 3b75c37b2aa0..c9c110dd1e62 100644 --- a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/ui/SparkConnectServerListenerSuite.scala +++ b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/ui/SparkConnectServerListenerSuite.scala @@ -37,7 +37,7 @@ class SparkConnectServerListenerSuite private var kvstore: ElementTrackingStore = _ - private val jobTag = ExecuteJobTag("sessionId", "userId", "operationId") + private val jobTag = ExecuteJobTag("userId", "sessionId", "operationId") after { if (kvstore != null) { @@ -174,7 +174,7 @@ class SparkConnectServerListenerSuite SparkListenerJobStart(0, System.currentTimeMillis(), Nil, createProperties)) listener.onOtherEvent( SparkListenerConnectSessionClosed("sessionId", "userId", System.currentTimeMillis())) -val exec = statusStore.getExecution(ExecuteJobTag("sessionId", "userId", "operationId")) +val exec = statusStore.getExecution(ExecuteJobTag("userId", "sessionId", "operationId")) assert(exec.isDefined) assert(exec.get.jobId === Seq("0")) assert(exec.get.sqlExecId === Set("0")) @@ -190,7 +190,7 @@ class SparkConnectServerListenerSuite listener.onOtherEvent(SparkListenerConnectSessionClosed(unknownSession, "userId", 0)) listener.onOtherEvent( SparkListenerConnectOperationStarted( -ExecuteJobTag("sessionId", "userId", "operationId"), +ExecuteJobTag("userId", "sessionId", "operationId"), "operationId", System.currentTimeMillis(), unknownSession, - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated (62dd64a5d13d -> 5a1559a7ef03)
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 62dd64a5d13d [SPARK-47583][CORE] SQL core: Migrate logError with variables to structured logging framework add 5a1559a7ef03 [SPARK-47921][CONNECT] Fix ExecuteJobTag creation in ExecuteHolder No new revisions were added by this update. Summary of changes: .../scala/org/apache/spark/sql/connect/service/ExecuteHolder.scala | 2 +- .../spark/sql/connect/ui/SparkConnectServerListenerSuite.scala | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated (7c81bdf1ed17 -> add49b3c115f)
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 7c81bdf1ed17 [SPARK-47345][SQL][TESTS] Xml functions suite add add49b3c115f [SPARK-47346][PYTHON] Make daemon mode configurable when creating Python planner workers No new revisions were added by this update. Summary of changes: .../src/main/scala/org/apache/spark/SparkEnv.scala | 27 ++ .../spark/api/python/PythonWorkerFactory.scala | 12 +- .../spark/api/python/StreamingPythonRunner.scala | 18 +-- .../api/python/PythonWorkerFactorySuite.scala | 2 +- .../sql/execution/python/PythonPlannerRunner.scala | 4 ++-- .../python/PythonStreamingSourceRunner.scala | 2 +- 6 files changed, 39 insertions(+), 26 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated (36df0a63a139 -> f028d3eba314)
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 36df0a63a139 [SPARK-47135][SS] Implement error classes for Kafka data loss exceptions add f028d3eba314 [SPARK-47214][PYTHON] Create UDTF API for 'analyze' method to differentiate constant NULL arguments and other types of arguments No new revisions were added by this update. Summary of changes: python/pyspark/sql/tests/test_udtf.py| 7 ++- python/pyspark/sql/udtf.py | 7 +++ python/pyspark/sql/worker/analyze_udtf.py| 9 ++--- .../expressions/FunctionTableSubqueryArgumentExpression.scala| 4 +--- 4 files changed, 20 insertions(+), 7 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated (e2dcb0274f13 -> f5d38001808a)
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from e2dcb0274f13 [SPARK-47173][SS][UI] Fix a typo in streaming UI explanation add f5d38001808a [SPARK-47079][PYTHON][SQL][CONNECT] Add Variant type info to PySpark No new revisions were added by this update. Summary of changes: .../src/main/protobuf/spark/connect/types.proto| 5 + .../connect/common/DataTypeProtoConverter.scala| 3 + .../spark/sql/connect/common/ProtoDataTypes.scala | 6 ++ python/pyspark/sql/connect/proto/types_pb2.py | 106 +++-- python/pyspark/sql/connect/proto/types_pb2.pyi | 26 + python/pyspark/sql/connect/types.py| 5 + python/pyspark/sql/tests/test_types.py | 7 ++ python/pyspark/sql/types.py| 36 +++ 8 files changed, 142 insertions(+), 52 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated (28951ed6681f -> 772a445e412b)
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 28951ed6681f [SPARK-47118][BUILD][CORE][SQL][UI] Migrate from Jetty 10 to Jetty 11 add 772a445e412b [SPARK-47035][SS][CONNECT] Protocol for Client-Side Listener No new revisions were added by this update. Summary of changes: .../src/main/protobuf/spark/connect/commands.proto | 37 + python/pyspark/sql/connect/proto/commands_pb2.py | 102 +++--- python/pyspark/sql/connect/proto/commands_pb2.pyi | 156 - 3 files changed, 246 insertions(+), 49 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated (6185e5cad7be -> 9debaeaa5a07)
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 6185e5cad7be [SPARK-47132][DOCS][PYTHON] Correct docstring for pyspark's dataframe.head add 9debaeaa5a07 [SPARK-47069][PYTHON][CONNECT] Introduce `spark.profile.show/dump` for SparkSession-based profiling No new revisions were added by this update. Summary of changes: python/pyspark/sql/connect/session.py | 30 ++ python/pyspark/sql/profiler.py | 70 ++ python/pyspark/sql/session.py | 33 ++ .../tests/connect/test_parity_memory_profiler.py | 2 +- .../sql/tests/connect/test_parity_udf_profiler.py | 2 +- python/pyspark/sql/tests/test_session.py | 63 ++- python/pyspark/sql/tests/test_udf_profiler.py | 28 - python/pyspark/tests/test_memory_profiler.py | 28 - 8 files changed, 171 insertions(+), 85 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-47002][PYTHON] Return better error message if UDTF 'analyze' method 'orderBy' field accidentally returns a list of strings
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 6569f15490ec [SPARK-47002][PYTHON] Return better error message if UDTF 'analyze' method 'orderBy' field accidentally returns a list of strings 6569f15490ec is described below commit 6569f15490ec144b495b777d650c42ee4cc184d3 Author: Daniel Tenedorio AuthorDate: Thu Feb 8 11:17:37 2024 -0800 [SPARK-47002][PYTHON] Return better error message if UDTF 'analyze' method 'orderBy' field accidentally returns a list of strings ### What changes were proposed in this pull request? This PR updates the Python UDTF API to check and return a better error message if the `analyze` method returns an `AnalyzeResult` object with an `orderBy` field erroneously set to a list of strings, rather than `OrderingColumn` instances. For example, this UDTF accidentally sets the `orderBy` field in this way: ``` from pyspark.sql.functions import AnalyzeResult, OrderingColumn, PartitioningColumn from pyspark.sql.types import IntegerType, Row, StructType class Udtf: def __init__(self): self._partition_col = None self._count = 0 self._sum = 0 self._last = None staticmethod def analyze(row: Row): return AnalyzeResult( schema=StructType() .add("user_id", IntegerType()) .add("count", IntegerType()) .add("total", IntegerType()) .add("last", IntegerType()), partitionBy=[ PartitioningColumn("user_id") ], orderBy=[ "timestamp" ], ) def eval(self, row: Row): self._partition_col = row["partition_col"] self._count += 1 self._last = row["input"] self._sum += row["input"] def terminate(self): yield self._partition_col, self._count, self._sum, self._last ``` ### Why are the changes needed? This improves error messages and helps keep users from getting confused. ### Does this PR introduce _any_ user-facing change? Yes, see above. ### How was this patch tested? This PR adds test coverage. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45062 from dtenedor/check-udtf-sort-columns. Authored-by: Daniel Tenedorio Signed-off-by: Takuya UESHIN --- python/pyspark/sql/worker/analyze_udtf.py | 48 ++-- .../sql-tests/analyzer-results/udtf/udtf.sql.out | 20 + .../test/resources/sql-tests/inputs/udtf/udtf.sql | 1 + .../resources/sql-tests/results/udtf/udtf.sql.out | 22 ++ .../apache/spark/sql/IntegratedUDFTestUtils.scala | 51 -- 5 files changed, 95 insertions(+), 47 deletions(-) diff --git a/python/pyspark/sql/worker/analyze_udtf.py b/python/pyspark/sql/worker/analyze_udtf.py index f61330b806cd..a4b26e0bdc61 100644 --- a/python/pyspark/sql/worker/analyze_udtf.py +++ b/python/pyspark/sql/worker/analyze_udtf.py @@ -31,7 +31,7 @@ from pyspark.serializers import ( write_with_length, SpecialLengths, ) -from pyspark.sql.functions import PartitioningColumn, SelectedColumn +from pyspark.sql.functions import OrderingColumn, PartitioningColumn, SelectedColumn from pyspark.sql.types import _parse_datatype_json_string, StructType from pyspark.sql.udtf import AnalyzeArgument, AnalyzeResult from pyspark.util import handle_worker_exception @@ -163,6 +163,18 @@ def main(infile: IO, outfile: IO) -> None: but the 'schema' field had the wrong type: {type(result.schema)}""" ) ) + +def invalid_analyze_result_field(field_name: str, expected_field: str) -> PySparkValueError: +return PySparkValueError( +format_error( +f""" +{error_prefix} because the static 'analyze' method returned an +'AnalyzeResult' object with the '{field_name}' field set to a value besides a +list or tuple of '{expected_field}' objects. Please update the table function +and then try the query again.""" +) +) + has_table_arg = any(arg.isTable for arg in args) or any( arg.isTable for arg in kwargs.values() ) @@ -190,32 +202,18
(spark) branch master updated: [SPARK-46688][SPARK-46691][PYTHON][CONNECT] Support v2 profiling in aggregate Pandas UDFs
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 9bd0d7c3ee13 [SPARK-46688][SPARK-46691][PYTHON][CONNECT] Support v2 profiling in aggregate Pandas UDFs 9bd0d7c3ee13 is described below commit 9bd0d7c3ee135036f5b370ff37517ae9d4d9f155 Author: Xinrong Meng AuthorDate: Wed Feb 7 13:50:09 2024 -0800 [SPARK-46688][SPARK-46691][PYTHON][CONNECT] Support v2 profiling in aggregate Pandas UDFs ### What changes were proposed in this pull request? Support v2 (perf, memory) profiling in Aggregate (Series to Scalar) Pandas UDFs, which rely on physical plan nodes AggregateInPandasExec and WindowInPandasExec. ### Why are the changes needed? Complete v2 profiling support. ### Does this PR introduce _any_ user-facing change? Yes. V2 profiling in Aggregate Pandas UDFs is supported. ### How was this patch tested? Unit tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45035 from xinrong-meng/other_p. Lead-authored-by: Xinrong Meng Co-authored-by: Takuya UESHIN Signed-off-by: Takuya UESHIN --- python/pyspark/sql/tests/test_udf_profiler.py | 61 ++ python/pyspark/tests/test_memory_profiler.py | 61 ++ .../execution/python/AggregateInPandasExec.scala | 3 +- .../sql/execution/python/WindowInPandasExec.scala | 2 +- 4 files changed, 124 insertions(+), 3 deletions(-) diff --git a/python/pyspark/sql/tests/test_udf_profiler.py b/python/pyspark/sql/tests/test_udf_profiler.py index 7e3d8e2dbe55..99719b5475c1 100644 --- a/python/pyspark/sql/tests/test_udf_profiler.py +++ b/python/pyspark/sql/tests/test_udf_profiler.py @@ -28,6 +28,7 @@ from typing import Iterator, cast from pyspark import SparkConf from pyspark.sql import SparkSession from pyspark.sql.functions import col, pandas_udf, udf +from pyspark.sql.window import Window from pyspark.profiler import UDFBasicProfiler from pyspark.testing.sqlutils import ( ReusedSQLTestCase, @@ -333,6 +334,66 @@ class UDFProfiler2TestsMixin: self.assertEqual(0, len(self.profile_results), str(self.profile_results.keys())) +@unittest.skipIf( +not have_pandas or not have_pyarrow, +cast(str, pandas_requirement_message or pyarrow_requirement_message), +) +def test_perf_profiler_pandas_udf_window(self): +# WindowInPandasExec +import pandas as pd + +@pandas_udf("double") +def mean_udf(v: pd.Series) -> float: +return v.mean() + +df = self.spark.createDataFrame( +[(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)], ("id", "v") +) +w = Window.partitionBy("id").orderBy("v").rowsBetween(-1, 0) + +with self.sql_conf({"spark.sql.pyspark.udf.profiler": "perf"}): +df.withColumn("mean_v", mean_udf("v").over(w)).show() + +self.assertEqual(1, len(self.profile_results), str(self.profile_results.keys())) + +for id in self.profile_results: +with self.trap_stdout() as io: +self.spark.showPerfProfiles(id) + +self.assertIn(f"Profile of UDF", io.getvalue()) +self.assertRegex( +io.getvalue(), f"5.*{os.path.basename(inspect.getfile(_do_computation))}" +) + +@unittest.skipIf( +not have_pandas or not have_pyarrow, +cast(str, pandas_requirement_message or pyarrow_requirement_message), +) +def test_perf_profiler_aggregate_in_pandas(self): +# AggregateInPandasExec +import pandas as pd + +@pandas_udf("double") +def min_udf(v: pd.Series) -> float: +return v.min() + +with self.sql_conf({"spark.sql.pyspark.udf.profiler": "perf"}): +df = self.spark.createDataFrame( +[(2, "Alice"), (3, "Alice"), (5, "Bob"), (10, "Bob")], ["age", "name"] +) +df.groupBy(df.name).agg(min_udf(df.age)).show() + +self.assertEqual(1, len(self.profile_results), str(self.profile_results.keys())) + +for id in self.profile_results: +with self.trap_stdout() as io: +self.spark.showPerfProfiles(id) + +self.assertIn(f"Profile of UDF", io.getvalue()) +self.assertRegex( +io.getvalue(), f"2.*{os.path.basename(inspect.getfile(_do_computation))}" +) + class UDFProfiler2Tests(UDFProfiler2TestsMixin, ReusedSQLTestCase): def setUp(self) -> None: diff --git a/python/pyspark
(spark) branch master updated: [SPARK-46966][PYTHON] Add UDTF API for 'analyze' method to indicate subset of input table columns to select
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 31f85e5ff77c [SPARK-46966][PYTHON] Add UDTF API for 'analyze' method to indicate subset of input table columns to select 31f85e5ff77c is described below commit 31f85e5ff77c9f9b704160c3e70849a488a4f40d Author: Daniel Tenedorio AuthorDate: Wed Feb 7 13:42:44 2024 -0800 [SPARK-46966][PYTHON] Add UDTF API for 'analyze' method to indicate subset of input table columns to select ### What changes were proposed in this pull request? This PR adds a UDTF API for the 'analyze' method to indicate subset of input table columns to select. For example, this UDTF populates this 'select' list to indicate that Spark should only return two input columns from the input table: 'input' and 'partition_col': ``` from pyspark.sql.functions import AnalyzeResult, OrderingColumn, PartitioningColumn, SelectedColumn from pyspark.sql.types import IntegerType, Row, StructType class Udtf: def __init__(self): self._partition_col = None self._count = 0 self._sum = 0 self._last = None staticmethod def analyze(row: Row): return AnalyzeResult( schema=StructType() .add("user_id", IntegerType()) .add("count", IntegerType()) .add("total", IntegerType()) .add("last", IntegerType()), partitionBy=[ PartitioningColumn("user_id") ], orderBy=[ OrderingColumn("timestamp") ], select=[ SelectedColumn("input"), SelectedColumn("partition_col") ]) def eval(self, row: Row): self._partition_col = row["partition_col"] self._count += 1 self._last = row["input"] self._sum += row["input"] def terminate(self): yield self._partition_col, self._count, self._sum, self._last ``` ### Why are the changes needed? This can reduce the amount of data sent between the JVM and Python interpreter, improving performance. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? This PR adds test coverage. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45007 from dtenedor/udtf-select-cols. Authored-by: Daniel Tenedorio Signed-off-by: Takuya UESHIN --- .../src/main/resources/error/error-classes.json| 6 ++ docs/sql-error-conditions.md | 6 ++ python/pyspark/sql/functions/builtin.py| 2 +- python/pyspark/sql/udtf.py | 32 ++- python/pyspark/sql/worker/analyze_udtf.py | 20 - .../FunctionTableSubqueryArgumentExpression.scala | 21 - .../spark/sql/catalyst/expressions/PythonUDF.scala | 27 +- .../spark/sql/errors/QueryCompilationErrors.scala | 6 ++ .../python/UserDefinedPythonFunction.scala | 15 +++- .../sql-tests/analyzer-results/udtf/udtf.sql.out | 79 + .../test/resources/sql-tests/inputs/udtf/udtf.sql | 6 ++ .../resources/sql-tests/results/udtf/udtf.sql.out | 98 ++ .../apache/spark/sql/IntegratedUDFTestUtils.scala | 62 -- 13 files changed, 367 insertions(+), 13 deletions(-) diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index e9dc715cf328..4fcf9248d3e2 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -3456,6 +3456,12 @@ ], "sqlState" : "42802" }, + "UDTF_INVALID_REQUESTED_SELECTED_EXPRESSION_FROM_ANALYZE_METHOD_REQUIRES_ALIAS" : { +"message" : [ + "Failed to evaluate the user-defined table function because its 'analyze' method returned a requested 'select' expression () that does not include a corresponding alias; please update the UDTF to specify an alias there and then try the query again." +], +"sqlState" : "42802" + }, "UNABLE_TO_ACQUIRE_MEMORY" : { "message" : [ "Unable to acquire bytes of memory, got ." diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md index 7c51a6bcc145
(spark) branch master updated: [SPARK-46687][PYTHON][CONNECT] Basic support of SparkSession-based memory profiler
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 528ac8b3e854 [SPARK-46687][PYTHON][CONNECT] Basic support of SparkSession-based memory profiler 528ac8b3e854 is described below commit 528ac8b3e8548a53d931007c36db3427c610f4da Author: Xinrong Meng AuthorDate: Mon Jan 29 13:08:17 2024 -0800 [SPARK-46687][PYTHON][CONNECT] Basic support of SparkSession-based memory profiler ### What changes were proposed in this pull request? Basic support of SparkSession-based memory profiler in both Spark Connect and non-Spark-Connect. ### Why are the changes needed? We need to make the memory profiler SparkSession-based to support memory profiling in Spark Connect. ### Does this PR introduce _any_ user-facing change? Yes, the SparkSession-based memory profiler is available. An example is as shown below ```py from pyspark.sql import SparkSession from pyspark.sql.functions import * from pyspark.taskcontext import TaskContext spark.conf.set("spark.sql.pyspark.udf.profiler", "memory") udf("string") def f(x): if TaskContext.get().partitionId() % 2 == 0: return str(x) else: return None spark.range(10).select(f(col("id"))).show() spark.showMemoryProfiles() ``` shows profile result: ``` Profile of UDF Filename: /var/folders/h_/60n1p_5s7751jx1st4_sk078gp/T/ipykernel_72839/2848225169.py Line #Mem usageIncrement Occurrences Line Contents = 7113.2 MiB113.2 MiB 10 udf("string") 8 def f(x): 9114.4 MiB 1.3 MiB 10 if TaskContext.get().partitionId() % 2 == 0: 10 31.8 MiB 0.1 MiB 4 return str(x) 11 else: 12 82.8 MiB 0.1 MiB 6 return None ``` ### How was this patch tested? New and existing unit tests: - pyspark.tests.test_memory_profiler - pyspark.sql.tests.connect.test_parity_memory_profiler And manual tests on Jupyter notebook. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44775 from xinrong-meng/memory_profiler_v2. Authored-by: Xinrong Meng Signed-off-by: Takuya UESHIN --- dev/sparktestsupport/modules.py| 1 + python/pyspark/profiler.py | 75 ++-- python/pyspark/sql/connect/session.py | 5 + python/pyspark/sql/session.py | 5 + .../tests/connect/test_parity_memory_profiler.py | 59 ++ python/pyspark/tests/test_memory_profiler.py | 212 - python/pyspark/worker.py | 36 +++- 7 files changed, 368 insertions(+), 25 deletions(-) diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index b9541c4be9b3..508cf56b9c87 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -1002,6 +1002,7 @@ pyspark_connect = Module( "pyspark.sql.tests.connect.test_parity_readwriter", "pyspark.sql.tests.connect.test_parity_udf", "pyspark.sql.tests.connect.test_parity_udf_profiler", +"pyspark.sql.tests.connect.test_parity_memory_profiler", "pyspark.sql.tests.connect.test_parity_udtf", "pyspark.sql.tests.connect.test_parity_pandas_udf", "pyspark.sql.tests.connect.test_parity_pandas_map", diff --git a/python/pyspark/profiler.py b/python/pyspark/profiler.py index b5f1bc4d714d..aa2288b36a02 100644 --- a/python/pyspark/profiler.py +++ b/python/pyspark/profiler.py @@ -19,6 +19,7 @@ from typing import ( Any, Callable, Dict, +Iterator, List, Optional, Tuple, @@ -37,7 +38,7 @@ import sys import warnings try: -from memory_profiler import choose_backend, CodeMap, LineProfiler # type: ignore[import] +from memory_profiler import CodeMap, LineProfiler # type: ignore[import] has_memory_profiler = True except Exception: @@ -196,16 +197,40 @@ if has_memory_profiler: for subcode in filter(inspect.iscode, code.co_consts): self.add(subcode, toplevel_code=toplevel_code) +class CodeMapForUDFV2(CodeMap): +def add( +self, +code: Any, +toplevel
(spark) branch master updated: [SPARK-46040][SQL][PYTHON] Update UDTF API for 'analyze' partitioning/ordering columns to support general expressions
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 91a02e5d9701 [SPARK-46040][SQL][PYTHON] Update UDTF API for 'analyze' partitioning/ordering columns to support general expressions 91a02e5d9701 is described below commit 91a02e5d97011f5f9b620c07b1c2f7d85291448b Author: Daniel Tenedorio AuthorDate: Mon Dec 4 16:51:56 2023 -0800 [SPARK-46040][SQL][PYTHON] Update UDTF API for 'analyze' partitioning/ordering columns to support general expressions ### What changes were proposed in this pull request? This PR updates the Python user-defined table function (UDTF) API for the `analyze` method to support general expressions for the `partitionBy` and `orderBy` fields of the `AnalyzeResult` class. For example, the following UDTF specifies to partition by `partition_col / 10` so that all rows with values of this column between 0-9 arrive in the same partition, then all rows with values between 10-19 in the next partition, and so on. ``` udtf class TestUDTF: def __init__(self): self._partition_col = None self._count = 0 self._sum = 0 self._last = None staticmethod def analyze(*args, **kwargs): return AnalyzeResult( schema=StructType() .add("partition_col", IntegerType()) .add("count", IntegerType()) .add("total", IntegerType()) .add("last", IntegerType()), partitionBy=[PartitioningColumn("partition_col / 10")], orderBy=[ OrderingColumn(name="input", ascending=True, overrideNullsFirst=False) ], ) def eval(self, row: Row): self._partition_col = row["partition_col"] self._count += 1 self._last = row["input"] if row["input"] is not None: self._sum += row["input"] def terminate(self): yield self._partition_col, self._count, self._sum, self._last ``` ### Why are the changes needed? This lets the UDTF partition by simple references to the columns of the input table just like before, but also gives the option to partition by general expressions based on those columns (just like the explicit `PARTITION BY` and `ORDER BY` clauses in the UDTF call in SQL). ### Does this PR introduce _any_ user-facing change? Yes, see above. ### How was this patch tested? This PR includes test coverage. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43946 from dtenedor/udtf-order-partition-by-exprs. Authored-by: Daniel Tenedorio Signed-off-by: Takuya UESHIN --- .../src/main/resources/error/error-classes.json| 6 +++ .../sql/connect/planner/SparkConnectPlanner.scala | 4 +- docs/sql-error-conditions.md | 6 +++ python/docs/source/user_guide/sql/python_udtf.rst | 2 +- python/pyspark/sql/udtf.py | 21 + .../spark/sql/errors/QueryCompilationErrors.scala | 7 +++ .../org/apache/spark/sql/UDTFRegistration.scala| 4 +- .../python/UserDefinedPythonFunction.scala | 53 - .../sql-tests/analyzer-results/udtf/udtf.sql.out | 47 +++ .../test/resources/sql-tests/inputs/udtf/udtf.sql | 3 ++ .../resources/sql-tests/results/udtf/udtf.sql.out | 54 ++ .../apache/spark/sql/IntegratedUDFTestUtils.scala | 39 +--- 12 files changed, 206 insertions(+), 40 deletions(-) diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index 9e0019b34728..6795ebcb0bd0 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -3110,6 +3110,12 @@ ], "sqlState" : "42802" }, + "UDTF_INVALID_ALIAS_IN_REQUESTED_ORDERING_STRING_FROM_ANALYZE_METHOD" : { +"message" : [ + "Failed to evaluate the user-defined table function because its 'analyze' method returned a requested OrderingColumn whose column name expression included an unnecessary alias ; please remove this alias and then try the query again." +], +"sqlState" : "42802" + }, "UNABLE_TO_ACQUIRE_MEMORY" : { "message" : [ "Unable to acquire bytes of memory, got ." diff --git a/connector/connect/server/src/main/scala/org/
(spark) branch master updated: [SPARK-45746][PYTHON] Return specific error messages if UDTF 'analyze' or 'eval' method accepts or returns wrong values
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new f5e4e84ce3a [SPARK-45746][PYTHON] Return specific error messages if UDTF 'analyze' or 'eval' method accepts or returns wrong values f5e4e84ce3a is described below commit f5e4e84ce3a7f65407d07cfc3eed2f51837527c1 Author: Daniel Tenedorio AuthorDate: Wed Nov 29 10:52:44 2023 -0800 [SPARK-45746][PYTHON] Return specific error messages if UDTF 'analyze' or 'eval' method accepts or returns wrong values ### What changes were proposed in this pull request? This PR adds checks to return specific error messages if any Python UDTF `analyze` or `eval` method accepts or returns wrong values. Error messages improved include: * If the `__init__` method takes more arguments than `self` and `analyze_result`. * If the UDTF call passes more or fewer arguments than `analyze` or `eval` expects (not using `*args` or `**kwargs`). * If the `analyze` method returns an object besides a `StructType` in the `AnalyzeResult` `schema` field. * If there are extra optional `AnalyzeResult` fields relating to input table arguments (e.g. `with_single_partition`) but the `analyze` method received no input table argument. * If the `analyze` method tries to return a list of strings for the `partition_by` optional field of the `AnalyzeResult` instead of a list of `PartitioningColumn` objects. * If the `AnalyzeResult` is missing the `schema` argument entirely. * If we use keyword arguments in the TVF call but the `analyze` or `eval` method does not accept arguments with those keyword(s) (or `**kwargs`). ### Why are the changes needed? This helps users understand how to easily fix their user-defined table functions if they are malformed. ### Does this PR introduce _any_ user-facing change? Yes, see above. ### How was this patch tested? This PR adds test coverage. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43611 from dtenedor/fix-more-udtf-errors. Authored-by: Daniel Tenedorio Signed-off-by: Takuya UESHIN --- python/pyspark/errors/error_classes.py | 15 + python/pyspark/sql/tests/test_udtf.py | 49 +-- python/pyspark/sql/worker/analyze_udtf.py | 86 +++- python/pyspark/worker.py | 50 ++- .../execution/python/BatchEvalPythonUDTFExec.scala | 2 + .../python/UserDefinedPythonFunction.scala | 7 +- .../sql-tests/analyzer-results/udtf/udtf.sql.out | 246 +++ .../test/resources/sql-tests/inputs/udtf/udtf.sql | 17 + .../resources/sql-tests/results/udtf/udtf.sql.out | 280 + .../apache/spark/sql/IntegratedUDFTestUtils.scala | 450 +++-- .../org/apache/spark/sql/SQLQueryTestSuite.scala | 20 +- .../sql/execution/python/PythonUDTFSuite.scala | 26 +- 12 files changed, 939 insertions(+), 309 deletions(-) diff --git a/python/pyspark/errors/error_classes.py b/python/pyspark/errors/error_classes.py index 6662efa8ca5..289b16c9b60 100644 --- a/python/pyspark/errors/error_classes.py +++ b/python/pyspark/errors/error_classes.py @@ -798,6 +798,21 @@ ERROR_CLASSES_JSON = """ "Cannot convert the output value of the column '' with type '' to the specified return type of the column: ''. Please check if the data types match and try again." ] }, + "UDTF_CONSTRUCTOR_INVALID_IMPLEMENTS_ANALYZE_METHOD" : { +"message" : [ + "Failed to evaluate the user-defined table function '' because its constructor is invalid: the function implements the 'analyze' method, but its constructor has more than two arguments (including the 'self' reference). Please update the table function so that its constructor accepts exactly one 'self' argument, or one 'self' argument plus another argument for the result of the 'analyze' method, and try the query again." +] + }, + "UDTF_CONSTRUCTOR_INVALID_NO_ANALYZE_METHOD" : { +"message" : [ + "Failed to evaluate the user-defined table function '' because its constructor is invalid: the function does not implement the 'analyze' method, and its constructor has more than one argument (including the 'self' reference). Please update the table function so that its constructor accepts exactly one 'self' argument, and try the query again." +] + }, + "UDTF_EVAL_METHOD_ARGUMENTS_DO_NOT_MATCH_SIGNATURE" : { +"message" : [ + "Failed to evalu
(spark) branch master updated: [SPARK-45810][PYTHON] Create Python UDTF API to stop consuming rows from the input table
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 36983443112 [SPARK-45810][PYTHON] Create Python UDTF API to stop consuming rows from the input table 36983443112 is described below commit 36983443112799dc2ee4462828e7c0552a63a229 Author: Daniel Tenedorio AuthorDate: Wed Nov 15 13:47:04 2023 -0800 [SPARK-45810][PYTHON] Create Python UDTF API to stop consuming rows from the input table ### What changes were proposed in this pull request? This PR creates a Python UDTF API to stop consuming rows from the input table. If the UDTF raises a `SkipRestOfInputTableException` exception in the `eval` method, then the UDTF stops consuming rows from the input table for that input partition, and finally calls the `terminate` method (if any) to represent a successful UDTF call. For example: ``` udtf(returnType="total: int") class TestUDTF: def __init__(self): self._total = 0 def eval(self, _: Row): self._total += 1 if self._total >= 3: raise SkipRestOfInputTableException("Stop at self._total >= 3") def terminate(self): yield self._total, ``` ### Why are the changes needed? This is useful when the UDTF logic knows that we don't have to scan the input table anymore, and skip the rest of the I/O for that case. ### Does this PR introduce _any_ user-facing change? Yes, see above. ### How was this patch tested? This PR adds test coverage. ### Was this patch authored or co-authored using generative AI tooling? No Closes #43682 from dtenedor/udtf-api-stop-consuming-input-rows. Authored-by: Daniel Tenedorio Signed-off-by: Takuya UESHIN --- python/docs/source/user_guide/sql/python_udtf.rst | 38 - python/pyspark/sql/functions.py | 1 + python/pyspark/sql/tests/test_udtf.py | 51 +++ python/pyspark/sql/udtf.py| 19 - python/pyspark/worker.py | 30 ++--- 5 files changed, 123 insertions(+), 16 deletions(-) diff --git a/python/docs/source/user_guide/sql/python_udtf.rst b/python/docs/source/user_guide/sql/python_udtf.rst index 0e0c6e28578..3e3c7634438 100644 --- a/python/docs/source/user_guide/sql/python_udtf.rst +++ b/python/docs/source/user_guide/sql/python_udtf.rst @@ -65,8 +65,8 @@ To implement a Python UDTF, you first need to define a class implementing the me def analyze(self, *args: Any) -> AnalyzeResult: """ -Computes the output schema of a particular call to this function in response to the -arguments provided. +Static method to compute the output schema of a particular call to this function in +response to the arguments provided. This method is optional and only needed if the registration of the UDTF did not provide a static output schema to be use for all calls to the function. In this context, @@ -101,12 +101,20 @@ To implement a Python UDTF, you first need to define a class implementing the me partitionBy: Sequence[PartitioningColumn] = field(default_factory=tuple) orderBy: Sequence[OrderingColumn] = field(default_factory=tuple) +Notes +- +- It is possible for the `analyze` method to accept the exact arguments expected, + mapping 1:1 with the arguments provided to the UDTF call. +- The `analyze` method can instead choose to accept positional arguments if desired + (using `*args`) or keyword arguments (using `**kwargs`). + Examples -analyze implementation that returns one output column for each word in the input string -argument. +This is an `analyze` implementation that returns one output column for each word in the +input string argument. ->>> def analyze(self, text: str) -> AnalyzeResult: +>>> @staticmethod +... def analyze(text: str) -> AnalyzeResult: ... schema = StructType() ... for index, word in enumerate(text.split(" ")): ... schema = schema.add(f"word_{index}") @@ -114,7 +122,8 @@ To implement a Python UDTF, you first need to define a class implementing the me Same as above, but using *args to accept the arguments. ->>> def analyze(self, *args) -> AnalyzeResult: +>>> @st
[spark] branch master updated: [SPARK-45523][PYTHON] Refactor the null-checking to have shortcuts
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 68c0f64dddc9 [SPARK-45523][PYTHON] Refactor the null-checking to have shortcuts 68c0f64dddc9 is described below commit 68c0f64dddc917be7d489f67fab06fcbfe500f0d Author: Takuya UESHIN AuthorDate: Tue Oct 24 13:12:11 2023 -0700 [SPARK-45523][PYTHON] Refactor the null-checking to have shortcuts ### What changes were proposed in this pull request? This is a follow-up of apache/spark#43356. Refactor the null-checking to have shortcuts. ### Why are the changes needed? The null-check can have shortcuts for some cases. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? The existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43492 from ueshin/issues/SPARK-45523/nullcheck. Authored-by: Takuya UESHIN Signed-off-by: Takuya UESHIN --- python/pyspark/worker.py | 211 +-- 1 file changed, 129 insertions(+), 82 deletions(-) diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index b1f59e1619fe..f6208032d9ac 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -24,7 +24,7 @@ import dataclasses import time from inspect import getfullargspec import json -from typing import Any, Callable, Iterable, Iterator +from typing import Any, Callable, Iterable, Iterator, Optional import faulthandler from pyspark.accumulators import _accumulatorRegistry @@ -58,7 +58,6 @@ from pyspark.sql.types import ( MapType, Row, StringType, -StructField, StructType, _create_row, _parse_datatype_json_string, @@ -700,7 +699,7 @@ def read_udtf(pickleSer, infile, eval_type): ) return_type = _parse_datatype_json_string(utf8_deserializer.loads(infile)) -if not type(return_type) == StructType: +if not isinstance(return_type, StructType): raise PySparkRuntimeError( f"The return type of a UDTF must be a struct type, but got {type(return_type)}." ) @@ -845,70 +844,112 @@ def read_udtf(pickleSer, infile, eval_type): "the query again." ) -# This determines which result columns have nullable types. -def check_nullable_column(i: int, data_type: DataType, nullable: bool) -> None: -if not nullable: -nullable_columns.add(i) -elif isinstance(data_type, ArrayType): -check_nullable_column(i, data_type.elementType, data_type.containsNull) -elif isinstance(data_type, StructType): -for subfield in data_type.fields: -check_nullable_column(i, subfield.dataType, subfield.nullable) -elif isinstance(data_type, MapType): -check_nullable_column(i, data_type.valueType, data_type.valueContainsNull) - -nullable_columns: set[int] = set() -for i, field in enumerate(return_type.fields): -check_nullable_column(i, field.dataType, field.nullable) - -# Compares each UDTF output row against the output schema for this particular UDTF call, -# raising an error if the two are incompatible. -def check_output_row_against_schema(row: Any, expected_schema: StructType) -> None: -for result_column_index in nullable_columns: - -def check_for_none_in_non_nullable_column( -value: Any, data_type: DataType, nullable: bool -) -> None: -if value is None and not nullable: -raise PySparkRuntimeError( -error_class="UDTF_EXEC_ERROR", -message_parameters={ -"method_name": "eval' or 'terminate", -"error": f"Column {result_column_index} within a returned row had a " -+ "value of None, either directly or within array/struct/map " -+ "subfields, but the corresponding column type was declared as " -+ "non-nullable; please update the UDTF to return a non-None value at " -+ "this location or otherwise declare the column type as nullable.", -}, -) -elif ( -isinstance(data_type, ArrayType) -and isinstance(value, list) -and not data_type.containsNull -): -for sub_value in value: -check_for_none_in_non_nullable_column( -
[spark] branch master updated (840306d867e2 -> 7ef96ee5b888)
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 840306d867e2 [SPARK-45643][CORE][SQL] Replace `s.c.mutable.MapOps#transform` with `s.c.mutable.MapOps#mapValuesInPlace` add 7ef96ee5b888 [SPARK-45524][PYTHON][SQL] Initial support for Python data source read API No new revisions were added by this update. Summary of changes: .../src/main/resources/error/error-classes.json| 6 + dev/sparktestsupport/modules.py| 1 + docs/sql-error-conditions.md | 6 + python/pyspark/errors/error_classes.py | 15 ++ python/pyspark/sql/datasource.py | 214 + python/pyspark/sql/tests/test_python_datasource.py | 60 ++ python/pyspark/sql/worker/plan_data_source_read.py | 155 +++ .../plans/logical/pythonLogicalOperators.scala | 40 +++- .../spark/sql/catalyst/trees/TreePatterns.scala| 1 + .../spark/sql/errors/QueryCompilationErrors.scala | 7 + .../spark/sql/execution/SparkOptimizer.scala | 8 +- .../spark/sql/execution/SparkStrategies.scala | 2 + .../datasources/PlanPythonDataSourceScan.scala | 90 + .../PythonDataSourcePartitionsExec.scala} | 66 ++- .../python/UserDefinedPythonDataSource.scala | 87 + .../apache/spark/sql/IntegratedUDFTestUtils.scala | 45 - .../execution/python/PythonDataSourceSuite.scala | 111 +++ 17 files changed, 864 insertions(+), 50 deletions(-) create mode 100644 python/pyspark/sql/datasource.py create mode 100644 python/pyspark/sql/tests/test_python_datasource.py create mode 100644 python/pyspark/sql/worker/plan_data_source_read.py create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PlanPythonDataSourceScan.scala copy sql/core/src/main/scala/org/apache/spark/sql/execution/{LocalTableScanExec.scala => python/PythonDataSourcePartitionsExec.scala} (58%) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonDataSource.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonDataSourceSuite.scala - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-45523][PYTHON] Return useful error message if UDTF returns None for any non-nullable column
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 227cd8bb8f5 [SPARK-45523][PYTHON] Return useful error message if UDTF returns None for any non-nullable column 227cd8bb8f5 is described below commit 227cd8bb8f5d8d442f225057db39591b3c630f46 Author: Daniel Tenedorio AuthorDate: Fri Oct 20 15:22:45 2023 -0700 [SPARK-45523][PYTHON] Return useful error message if UDTF returns None for any non-nullable column ### What changes were proposed in this pull request? This PR updates Python UDTF evaluation to return a useful error message if UDTF returns None for any non-nullable column. This implementation also checks recursively for None values in subfields of array/struct/map columns as well. For example: ``` from pyspark.sql.functions import AnalyzeResult from pyspark.sql.types import ArrayType, IntegerType, StringType, StructType class Tvf: staticmethod def analyze(*args): return AnalyzeResult( schema=StructType() .add("result", ArrayType(IntegerType(), containsNull=False), True) ) def eval(self, *args): yield [1, 2, 3, 4], def terminate(self): yield [1, 2, None, 3], ``` ``` SELECT * FROM Tvf(TABLE(VALUES (0), (1))) > org.apache.spark.api.python.PythonException [UDTF_EXEC_ERROR] User defined table function encountered an error in the 'eval' or 'terminate' method: Column 0 within a returned row had a value of None, either directly or within array/struct/map subfields, but the corresponding column type was declared as non nullable; please update the UDTF to return a non-None value at this location or otherwise declare the column type as nullable. ``` ### Why are the changes needed? Previously this case returned a null pointer exception. ### Does this PR introduce _any_ user-facing change? Yes, see above. ### How was this patch tested? This PR adds new test coverage. ### Was this patch authored or co-authored using generative AI tooling? No Closes #43356 from dtenedor/improve-errors-null-checks. Authored-by: Daniel Tenedorio Signed-off-by: Takuya UESHIN --- python/pyspark/worker.py | 72 + .../sql-tests/analyzer-results/udtf/udtf.sql.out | 60 .../test/resources/sql-tests/inputs/udtf/udtf.sql | 12 + .../resources/sql-tests/results/udtf/udtf.sql.out | 90 ++ .../apache/spark/sql/IntegratedUDFTestUtils.scala | 359 - .../org/apache/spark/sql/SQLQueryTestSuite.scala | 43 ++- 6 files changed, 626 insertions(+), 10 deletions(-) diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index df7dd1bc2f7..b1f59e1619f 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -52,9 +52,13 @@ from pyspark.sql.pandas.serializers import ( ) from pyspark.sql.pandas.types import to_arrow_type from pyspark.sql.types import ( +ArrayType, BinaryType, +DataType, +MapType, Row, StringType, +StructField, StructType, _create_row, _parse_datatype_json_string, @@ -841,6 +845,71 @@ def read_udtf(pickleSer, infile, eval_type): "the query again." ) +# This determines which result columns have nullable types. +def check_nullable_column(i: int, data_type: DataType, nullable: bool) -> None: +if not nullable: +nullable_columns.add(i) +elif isinstance(data_type, ArrayType): +check_nullable_column(i, data_type.elementType, data_type.containsNull) +elif isinstance(data_type, StructType): +for subfield in data_type.fields: +check_nullable_column(i, subfield.dataType, subfield.nullable) +elif isinstance(data_type, MapType): +check_nullable_column(i, data_type.valueType, data_type.valueContainsNull) + +nullable_columns: set[int] = set() +for i, field in enumerate(return_type.fields): +check_nullable_column(i, field.dataType, field.nullable) + +# Compares each UDTF output row against the output schema for this particular UDTF call, +# raising an error if the two are incompatible. +def check_output_row_against_schema(row: Any, expected_schema: StructType) -> None: +for result_column_index in nullable_columns: + +def check_for_none_in_non_nullable_column( +value: Any, data_type: DataType, nullable: bool +) -> None: +if value is None and not nullable: +raise PySpa
[spark] branch master updated (b10fea96b5b -> 4a0ed9cd725)
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from b10fea96b5b [SPARK-45566][PS] Support Pandas-like testing utils for Pandas API on Spark add 4a0ed9cd725 [SPARK-45577][PYTHON] Fix UserDefinedPythonTableFunctionAnalyzeRunner to pass folded values from named arguments No new revisions were added by this update. Summary of changes: python/pyspark/sql/tests/test_udtf.py | 17 +++-- .../execution/python/UserDefinedPythonFunction.scala| 12 2 files changed, 23 insertions(+), 6 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated (f72b87b90be -> 12880c846b5)
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from f72b87b90be [SPARK-45515][CORE][SQL] Use enhanced `switch` expressions to replace the regular `switch` statement add 12880c846b5 [SPARK-45266][PYTHON][FOLLOWUP] Fix to resolve UnresolvedPolymorphicPythonUDTF when the table argument is specified as a named argument No new revisions were added by this update. Summary of changes: python/pyspark/sql/tests/test_udtf.py | 272 - .../spark/sql/catalyst/analysis/Analyzer.scala | 2 + 2 files changed, 156 insertions(+), 118 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-45505][PYTHON] Refactor analyzeInPython to make it reusable
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 280f6b33110d [SPARK-45505][PYTHON] Refactor analyzeInPython to make it reusable 280f6b33110d is described below commit 280f6b33110d707ebee6fec6e5bafa45b45213ae Author: allisonwang-db AuthorDate: Thu Oct 12 17:02:41 2023 -0700 [SPARK-45505][PYTHON] Refactor analyzeInPython to make it reusable ### What changes were proposed in this pull request? Currently, the `analyzeInPython` method in UserDefinedPythonTableFunction object can starts a Python process in driver and run a Python function in the Python process. This PR aims to refactor this logic into a reusable runner class. ### Why are the changes needed? To make the code more reusable. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing tests ### Was this patch authored or co-authored using generative AI tooling? No Closes #43340 from allisonwang-db/spark-45505-refactor-analyze-in-py. Authored-by: allisonwang-db Signed-off-by: Takuya UESHIN --- python/pyspark/sql/worker/analyze_udtf.py | 6 +- .../org/apache/spark/sql/internal/SQLConf.scala| 18 +- .../sql/execution/python/PythonPlannerRunner.scala | 177 .../python/UserDefinedPythonFunction.scala | 321 +++-- 4 files changed, 286 insertions(+), 236 deletions(-) diff --git a/python/pyspark/sql/worker/analyze_udtf.py b/python/pyspark/sql/worker/analyze_udtf.py index a6aa381eb14a..9e84b880fc96 100644 --- a/python/pyspark/sql/worker/analyze_udtf.py +++ b/python/pyspark/sql/worker/analyze_udtf.py @@ -98,14 +98,14 @@ def main(infile: IO, outfile: IO) -> None: """ Runs the Python UDTF's `analyze` static method. -This process will be invoked from `UserDefinedPythonTableFunction.analyzeInPython` in JVM -and receive the Python UDTF and its arguments for the `analyze` static method, +This process will be invoked from `UserDefinedPythonTableFunctionAnalyzeRunner.runInPython` +in JVM and receive the Python UDTF and its arguments for the `analyze` static method, and call the `analyze` static method, and send back a AnalyzeResult as a result of the method. """ try: check_python_version(infile) -memory_limit_mb = int(os.environ.get("PYSPARK_UDTF_ANALYZER_MEMORY_MB", "-1")) +memory_limit_mb = int(os.environ.get("PYSPARK_PLANNER_MEMORY_MB", "-1")) setup_memory_limits(memory_limit_mb) setup_spark_files(infile) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 12ec9e911d31..000694f6f1bb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -3008,14 +3008,14 @@ object SQLConf { .booleanConf .createWithDefault(false) - val PYTHON_TABLE_UDF_ANALYZER_MEMORY = -buildConf("spark.sql.analyzer.pythonUDTF.analyzeInPython.memory") - .doc("The amount of memory to be allocated to PySpark for Python UDTF analyzer, in MiB " + -"unless otherwise specified. If set, PySpark memory for Python UDTF analyzer will be " + -"limited to this amount. If not set, Spark will not limit Python's " + -"memory use and it is up to the application to avoid exceeding the overhead memory space " + -"shared with other non-JVM processes.\nNote: Windows does not support resource limiting " + -"and actual resource is not limited on MacOS.") + val PYTHON_PLANNER_EXEC_MEMORY = +buildConf("spark.sql.planner.pythonExecution.memory") + .doc("Specifies the memory allocation for executing Python code in Spark driver, in MiB. " + +"When set, it caps the memory for Python execution to the specified amount. " + +"If not set, Spark will not limit Python's memory usage and it is up to the application " + +"to avoid exceeding the overhead memory space shared with other non-JVM processes.\n" + +"Note: Windows does not support resource limiting and actual resource is not limited " + +"on MacOS.") .version("4.0.0") .bytesConf(ByteUnit.MiB) .createOptional @@ -5157,7 +5157,7 @@ class SQLConf extends Serializable with Logging with SqlApiConf { def pysparkWorkerPythonExecutable: Option[String] =
[spark] branch master updated: [SPARK-45402][SQL][PYTHON] Add UDTF API for 'eval' and 'terminate' methods to consume previous 'analyze' result
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 69cf80d25f0e [SPARK-45402][SQL][PYTHON] Add UDTF API for 'eval' and 'terminate' methods to consume previous 'analyze' result 69cf80d25f0e is described below commit 69cf80d25f0e4ed46ec38a63e063471988c31732 Author: Daniel Tenedorio AuthorDate: Wed Oct 11 18:52:06 2023 -0700 [SPARK-45402][SQL][PYTHON] Add UDTF API for 'eval' and 'terminate' methods to consume previous 'analyze' result ### What changes were proposed in this pull request? This PR adds a Python UDTF API for the `eval` and `terminate` methods to consume the previous `analyze` result. This also works for subclasses of the `AnalyzeResult` class, allowing the UDTF to return custom state from `analyze` to be consumed later. For example, we can now define a UDTF that perform complex initialization in the `analyze` method and then returns the result of that in the `terminate` method: ``` def MyUDTF(self): dataclass class AnalyzeResultWithBuffer(AnalyzeResult): buffer: str udtf class TestUDTF: def __init__(self, analyze_result): self._total = 0 self._buffer = do_complex_initialization(analyze_result.buffer) staticmethod def analyze(argument, _): return AnalyzeResultWithBuffer( schema=StructType() .add("total", IntegerType()) .add("buffer", StringType()), with_single_partition=True, buffer=argument.value, ) def eval(self, argument, row: Row): self._total += 1 def terminate(self): yield self._total, self._buffer self.spark.udtf.register("my_ddtf", MyUDTF) ``` Then the results might look like: ``` sql( """ WITH t AS ( SELECT id FROM range(1, 21) ) SELECT total, buffer FROM test_udtf("abc", TABLE(t)) """ ).collect() > 20, "complex_initialization_result" ``` ### Why are the changes needed? In this way, the UDTF can perform potentially expensive initialization logic in the `analyze` method just once and result the result of such initialization rather than repeating the initialization in `eval`. ### Does this PR introduce _any_ user-facing change? Yes, see above. ### How was this patch tested? This PR adds new unit test coverage. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43204 from dtenedor/prepare-string. Authored-by: Daniel Tenedorio Signed-off-by: Takuya UESHIN --- python/docs/source/user_guide/sql/python_udtf.rst | 124 - python/pyspark/sql/tests/test_udtf.py | 53 + python/pyspark/sql/udtf.py | 5 +- python/pyspark/sql/worker/analyze_udtf.py | 2 + python/pyspark/worker.py | 34 +- .../spark/sql/catalyst/analysis/Analyzer.scala | 5 +- .../spark/sql/catalyst/expressions/PythonUDF.scala | 20 +++- .../execution/python/BatchEvalPythonUDTFExec.scala | 8 ++ .../python/UserDefinedPythonFunction.scala | 7 +- .../sql-tests/analyzer-results/udtf/udtf.sql.out | 26 +++-- .../test/resources/sql-tests/inputs/udtf/udtf.sql | 9 +- .../resources/sql-tests/results/udtf/udtf.sql.out | 28 +++-- .../apache/spark/sql/IntegratedUDFTestUtils.scala | 64 ++- .../sql/execution/python/PythonUDTFSuite.scala | 42 +-- 14 files changed, 374 insertions(+), 53 deletions(-) diff --git a/python/docs/source/user_guide/sql/python_udtf.rst b/python/docs/source/user_guide/sql/python_udtf.rst index 74d8eb889861..fb42644dc702 100644 --- a/python/docs/source/user_guide/sql/python_udtf.rst +++ b/python/docs/source/user_guide/sql/python_udtf.rst @@ -50,10 +50,108 @@ To implement a Python UDTF, you first need to define a class implementing the me Notes - -- This method does not accept any extra arguments. Only the default - constructor is supported. - You cannot create or reference the Spark session within the UDTF. Any attempt to do so will result in a serialization error. +- If the below `analyze` method is implemented, it is also possible to define this + method as: `__init__(self, analyze_result:
[spark] branch master updated: [SPARK-45362][PYTHON] Project out PARTITION BY expressions before Python UDTF 'eval' method consumes them
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new b3d5bc0c109 [SPARK-45362][PYTHON] Project out PARTITION BY expressions before Python UDTF 'eval' method consumes them b3d5bc0c109 is described below commit b3d5bc0c10908aa66510844eaabc43b6764dd7c0 Author: Daniel Tenedorio AuthorDate: Thu Sep 28 14:02:46 2023 -0700 [SPARK-45362][PYTHON] Project out PARTITION BY expressions before Python UDTF 'eval' method consumes them ### What changes were proposed in this pull request? This PR projects out PARTITION BY expressions before Python UDTF 'eval' method consumes them. Before this PR, if a query included this `PARTITION BY` clause: ``` SELECT * FROM udtf((SELECT a, b FROM TABLE t) PARTITION BY (c, d)) ``` Then the `eval` method received four columns in each row: `a, b, c, d`. After this PR, the `eval` method only receives two columns: `a, b`, as expected. ### Why are the changes needed? This makes the Python UDTF `TABLE` columns consistently match what the `eval` method receives, as expected. ### Does this PR introduce _any_ user-facing change? Yes, see above. ### How was this patch tested? This PR adds new unit tests. ### Was this patch authored or co-authored using generative AI tooling? No Closes #43156 from dtenedor/project-out-partition-exprs. Authored-by: Daniel Tenedorio Signed-off-by: Takuya UESHIN --- python/pyspark/sql/tests/test_udtf.py | 12 python/pyspark/worker.py | 31 +++ 2 files changed, 39 insertions(+), 4 deletions(-) diff --git a/python/pyspark/sql/tests/test_udtf.py b/python/pyspark/sql/tests/test_udtf.py index 97d5190a506..a1d82056c50 100644 --- a/python/pyspark/sql/tests/test_udtf.py +++ b/python/pyspark/sql/tests/test_udtf.py @@ -2009,6 +2009,10 @@ class BaseUDTFTestsMixin: self._partition_col = None def eval(self, row: Row): +# Make sure that the PARTITION BY expressions were projected out. +assert len(row.asDict().items()) == 2 +assert "partition_col" in row +assert "input" in row self._sum += row["input"] if self._partition_col is not None and self._partition_col != row["partition_col"]: # Make sure that all values of the partitioning column are the same @@ -2092,6 +2096,10 @@ class BaseUDTFTestsMixin: self._partition_col = None def eval(self, row: Row, partition_col: str): +# Make sure that the PARTITION BY and ORDER BY expressions were projected out. +assert len(row.asDict().items()) == 2 +assert "partition_col" in row +assert "input" in row # Make sure that all values of the partitioning column are the same # for each row consumed by this method for this instance of the class. if self._partition_col is not None and self._partition_col != row[partition_col]: @@ -2247,6 +2255,10 @@ class BaseUDTFTestsMixin: ) def eval(self, row: Row): +# Make sure that the PARTITION BY and ORDER BY expressions were projected out. +assert len(row.asDict().items()) == 2 +assert "partition_col" in row +assert "input" in row # Make sure that all values of the partitioning column are the same # for each row consumed by this method for this instance of the class. if self._partition_col is not None and self._partition_col != row["partition_col"]: diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 77481704979..4cffb02a64a 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -51,7 +51,14 @@ from pyspark.sql.pandas.serializers import ( ApplyInPandasWithStateSerializer, ) from pyspark.sql.pandas.types import to_arrow_type -from pyspark.sql.types import BinaryType, Row, StringType, StructType, _parse_datatype_json_string +from pyspark.sql.types import ( +BinaryType, +Row, +StringType, +StructType, +_create_row, +_parse_datatype_json_string, +) from pyspark.util import fail_on_stopiteration, handle_worker_exception from pyspark import shuffle from pyspark.errors import PySparkRuntimeError, PySparkTypeError @@ -735,7 +742,12 @@ def read_udtf(pickleSer, infile, eval_type): yield row self._udtf = se
[spark] branch master updated: [SPARK-45266][PYTHON] Refactor ResolveFunctions analyzer rule to delay making lateral join when table arguments are used
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 2cf937f9bac [SPARK-45266][PYTHON] Refactor ResolveFunctions analyzer rule to delay making lateral join when table arguments are used 2cf937f9bac is described below commit 2cf937f9bac2131f3657660a8d65d07ab4ece490 Author: Takuya UESHIN AuthorDate: Thu Sep 28 10:37:18 2023 -0700 [SPARK-45266][PYTHON] Refactor ResolveFunctions analyzer rule to delay making lateral join when table arguments are used ### What changes were proposed in this pull request? Refactors `ResolveFunctions` analyzer rule to delay making lateral join when table arguments are used. - Delay making lateral join when table arguments are used to after all the children are resolved - Resolve `UnresolvedPolymorphicPythonUDTF` in one place - Introduce a new error class `UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.UNSUPPORTED_TABLE_ARGUMENT` if table arguments are used inproperly. ### Why are the changes needed? The analyzer rule `ResolveFunctions` became complicated. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43042 from ueshin/issues/SPARK-45266/analyzer. Authored-by: Takuya UESHIN Signed-off-by: Takuya UESHIN --- .../src/main/resources/error/error-classes.json| 5 + ...ted-subquery-expression-category-error-class.md | 4 + .../spark/sql/catalyst/analysis/Analyzer.scala | 155 + .../sql/catalyst/analysis/CheckAnalysis.scala | 5 + .../spark/sql/catalyst/expressions/PythonUDF.scala | 6 +- .../named-function-arguments.sql.out | 16 +-- .../results/named-function-arguments.sql.out | 16 +-- .../sql/execution/python/PythonUDTFSuite.scala | 20 ++- 8 files changed, 103 insertions(+), 124 deletions(-) diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index 0882e387176..58fcedae332 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -3484,6 +3484,11 @@ "message" : [ "IN/EXISTS predicate subqueries can only be used in filters, joins, aggregations, window functions, projections, and UPDATE/MERGE/DELETE commands." ] + }, + "UNSUPPORTED_TABLE_ARGUMENT" : { +"message" : [ + "Table arguments are used in a function where they are not supported." +] } }, "sqlState" : "0A000" diff --git a/docs/sql-error-conditions-unsupported-subquery-expression-category-error-class.md b/docs/sql-error-conditions-unsupported-subquery-expression-category-error-class.md index f61ea721aa0..45ad386c666 100644 --- a/docs/sql-error-conditions-unsupported-subquery-expression-category-error-class.md +++ b/docs/sql-error-conditions-unsupported-subquery-expression-category-error-class.md @@ -73,4 +73,8 @@ Correlated scalar subqueries can only be used in filters, aggregations, projecti IN/EXISTS predicate subqueries can only be used in filters, joins, aggregations, window functions, projections, and UPDATE/MERGE/DELETE commands``. +## UNSUPPORTED_TABLE_ARGUMENT + +Table arguments are used in a function where they are not supported``. + diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 67a958d73f7..cc0bfd3fc31 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -2082,7 +2082,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor case u: UnresolvedTableValuedFunction if u.functionArgs.forall(_.resolved) => withPosition(u) { try { -val resolvedTvf = resolveBuiltinOrTempTableFunction(u.name, u.functionArgs).getOrElse { +val resolvedFunc = resolveBuiltinOrTempTableFunction(u.name, u.functionArgs).getOrElse { val CatalogAndIdentifier(catalog, ident) = expandIdentifier(u.name) if (CatalogV2Util.isSessionCatalog(catalog)) { v1SessionCatalog.resolvePersistentTableFunction( @@ -2092,93 +2092,19 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor catalog, "table-valued functions") } }
[spark] branch master updated: [SPARK-45118][PYTHON] Refactor converters for complex types to short cut when the element types don't need converters
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 090fd18f362 [SPARK-45118][PYTHON] Refactor converters for complex types to short cut when the element types don't need converters 090fd18f362 is described below commit 090fd18f36242857a8d7b81ef78428775c1d1e42 Author: Takuya UESHIN AuthorDate: Thu Sep 14 10:44:07 2023 -0700 [SPARK-45118][PYTHON] Refactor converters for complex types to short cut when the element types don't need converters ### What changes were proposed in this pull request? Refactors converters for complex types to short cut when the element types don't need converters. The following refactors are done in this PR: - Provide a shortcut when the element types in complex types don't need converters - Check `None`s before calling the converter - Remove extra type checks just for assertions ### Why are the changes needed? When the element types in complex types don't need converters, we can provide a shortcut to avoid extra function calls. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added related tests and existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #42874 from ueshin/issues/SPARK-45118/converters. Authored-by: Takuya UESHIN Signed-off-by: Takuya UESHIN --- dev/sparktestsupport/modules.py | 1 + python/pyspark/sql/pandas/types.py| 442 ++-- python/pyspark/sql/tests/pandas/test_converter.py | 595 ++ 3 files changed, 886 insertions(+), 152 deletions(-) diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index 63cbbe6003d..0a751052491 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -491,6 +491,7 @@ pyspark_sql = Module( "pyspark.sql.tests.pandas.test_pandas_udf_typehints", "pyspark.sql.tests.pandas.test_pandas_udf_typehints_with_future_annotations", "pyspark.sql.tests.pandas.test_pandas_udf_window", +"pyspark.sql.tests.pandas.test_converter", "pyspark.sql.tests.test_pandas_sqlmetrics", "pyspark.sql.tests.test_readwriter", "pyspark.sql.tests.test_serde", diff --git a/python/pyspark/sql/pandas/types.py b/python/pyspark/sql/pandas/types.py index b02a003e632..54cd6fa7016 100644 --- a/python/pyspark/sql/pandas/types.py +++ b/python/pyspark/sql/pandas/types.py @@ -47,7 +47,6 @@ from pyspark.sql.types import ( NullType, DataType, UserDefinedType, -Row, _create_row, ) from pyspark.errors import PySparkTypeError, UnsupportedOperationException @@ -580,15 +579,21 @@ def _create_converter_to_pandas( if _ndarray_as_list: if _element_conv is None: -_element_conv = lambda x: x # noqa: E731 -def convert_array_ndarray_as_list(value: Any) -> Any: -if value is None: -return None -else: +def convert_array_ndarray_as_list(value: Any) -> Any: # In Arrow Python UDF, ArrayType is converted to `np.ndarray` # whereas a list is expected. -return [_element_conv(v) for v in value] # type: ignore[misc] +return list(value) + +else: + +def convert_array_ndarray_as_list(value: Any) -> Any: +# In Arrow Python UDF, ArrayType is converted to `np.ndarray` +# whereas a list is expected. +return [ +_element_conv(v) if v is not None else None # type: ignore[misc] +for v in value +] return convert_array_ndarray_as_list else: @@ -596,34 +601,53 @@ def _create_converter_to_pandas( return None def convert_array_ndarray_as_ndarray(value: Any) -> Any: -if value is None: -return None -elif isinstance(value, np.ndarray): +if isinstance(value, np.ndarray): # `pyarrow.Table.to_pandas` uses `np.ndarray`. -return np.array([_element_conv(v) for v in value]) # type: ignore[misc] +return np.array( +[ +_elemen
[spark] branch master updated: [SPARK-44640][PYTHON][FOLLOW-UP] Update UDTF error messages to include method name
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 3e22c8653d7 [SPARK-44640][PYTHON][FOLLOW-UP] Update UDTF error messages to include method name 3e22c8653d7 is described below commit 3e22c8653d728a6b8523051faddcca437accfc22 Author: allisonwang-db AuthorDate: Sat Sep 2 16:07:09 2023 -0700 [SPARK-44640][PYTHON][FOLLOW-UP] Update UDTF error messages to include method name ### What changes were proposed in this pull request? This PR is a follow-up for SPARK-44640 to make the error message of a few UDTF errors more informative by including the method name in the error message (`eval` or `terminate`). ### Why are the changes needed? To improve error messages. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No Closes #42726 from allisonwang-db/SPARK-44640-follow-up. Authored-by: allisonwang-db Signed-off-by: Takuya UESHIN --- python/pyspark/errors/error_classes.py | 8 python/pyspark/sql/tests/test_udtf.py | 21 +++ python/pyspark/worker.py | 37 +- 3 files changed, 52 insertions(+), 14 deletions(-) diff --git a/python/pyspark/errors/error_classes.py b/python/pyspark/errors/error_classes.py index ca448a169e8..74f52c416e9 100644 --- a/python/pyspark/errors/error_classes.py +++ b/python/pyspark/errors/error_classes.py @@ -244,7 +244,7 @@ ERROR_CLASSES_JSON = """ }, "INVALID_ARROW_UDTF_RETURN_TYPE" : { "message" : [ - "The return type of the arrow-optimized Python UDTF should be of type 'pandas.DataFrame', but the function returned a value of type with value: ." + "The return type of the arrow-optimized Python UDTF should be of type 'pandas.DataFrame', but the '' method returned a value of type with value: ." ] }, "INVALID_BROADCAST_OPERATION": { @@ -745,17 +745,17 @@ ERROR_CLASSES_JSON = """ }, "UDTF_INVALID_OUTPUT_ROW_TYPE" : { "message" : [ -"The type of an individual output row in the UDTF is invalid. Each row should be a tuple, list, or dict, but got ''. Please make sure that the output rows are of the correct type." +"The type of an individual output row in the '' method of the UDTF is invalid. Each row should be a tuple, list, or dict, but got ''. Please make sure that the output rows are of the correct type." ] }, "UDTF_RETURN_NOT_ITERABLE" : { "message" : [ - "The return value of the UDTF is invalid. It should be an iterable (e.g., generator or list), but got ''. Please make sure that the UDTF returns one of these types." + "The return value of the '' method of the UDTF is invalid. It should be an iterable (e.g., generator or list), but got ''. Please make sure that the UDTF returns one of these types." ] }, "UDTF_RETURN_SCHEMA_MISMATCH" : { "message" : [ - "The number of columns in the result does not match the specified schema. Expected column count: , Actual column count: . Please make sure the values returned by the function have the same number of columns as specified in the output schema." + "The number of columns in the result does not match the specified schema. Expected column count: , Actual column count: . Please make sure the values returned by the '' method have the same number of columns as specified in the output schema." ] }, "UDTF_RETURN_TYPE_MISMATCH" : { diff --git a/python/pyspark/sql/tests/test_udtf.py b/python/pyspark/sql/tests/test_udtf.py index c5f8b7693c2..97d5190a506 100644 --- a/python/pyspark/sql/tests/test_udtf.py +++ b/python/pyspark/sql/tests/test_udtf.py @@ -190,6 +190,27 @@ class BaseUDTFTestsMixin: with self.assertRaisesRegex(PythonException, "UDTF_RETURN_NOT_ITERABLE"): TestUDTF(lit(1)).collect() +def test_udtf_with_zero_arg_and_invalid_return_value(self): +@udtf(returnType="x: int") +class TestUDTF: +def eval(self): +return 1 + +with self.assertRaisesRegex(PythonException, "UDTF_RETURN_NOT_ITERABLE"): +TestUDTF().collect() + +def test_udtf_with_invalid_return_value_in_terminate(self): +@udtf(returnType="x: int") +class TestUDTF: +def eval(self,
[spark] branch master updated (df534c355d9 -> e4ebb372fa1)
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from df534c355d9 [SPARK-44952][SQL][PYTHON] Support named arguments in aggregate Pandas UDFs add e4ebb372fa1 [SPARK-44901][SQL] Add API in Python UDTF 'analyze' method to return partitioning/ordering expressions No new revisions were added by this update. Summary of changes: .../src/main/resources/error/error-classes.json| 12 + docs/sql-error-conditions.md | 12 + python/pyspark/sql/functions.py| 1 + python/pyspark/sql/tests/test_udtf.py | 142 ++- python/pyspark/sql/udtf.py | 44 +- python/pyspark/sql/worker/analyze_udtf.py | 18 + .../spark/sql/catalyst/analysis/Analyzer.scala | 53 ++- .../spark/sql/catalyst/expressions/PythonUDF.scala | 100 - .../sql/catalyst/expressions/generators.scala | 2 +- .../spark/sql/errors/QueryCompilationErrors.scala | 22 + .../python/UserDefinedPythonFunction.scala | 49 ++- .../sql-tests/analyzer-results/udtf/udtf.sql.out | 397 ++ .../test/resources/sql-tests/inputs/udtf/udtf.sql | 91 + .../resources/sql-tests/results/udtf/udtf.sql.out | 448 + .../apache/spark/sql/IntegratedUDFTestUtils.scala | 272 - .../org/apache/spark/sql/SQLQueryTestSuite.scala | 40 +- .../sql/execution/python/PythonUDTFSuite.scala | 68 +++- 17 files changed, 1716 insertions(+), 55 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-44952][SQL][PYTHON] Support named arguments in aggregate Pandas UDFs
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new df534c355d9 [SPARK-44952][SQL][PYTHON] Support named arguments in aggregate Pandas UDFs df534c355d9 is described below commit df534c355d9059fb5b128491a8f037baa121cbd7 Author: Takuya UESHIN AuthorDate: Fri Sep 1 10:41:37 2023 -0700 [SPARK-44952][SQL][PYTHON] Support named arguments in aggregate Pandas UDFs ### What changes were proposed in this pull request? Supports named arguments in aggregate Pandas UDFs. For example: ```py >>> pandas_udf("double") ... def weighted_mean(v: pd.Series, w: pd.Series) -> float: ... import numpy as np ... return np.average(v, weights=w) ... >>> df = spark.createDataFrame( ... [(1, 1.0, 1.0), (1, 2.0, 2.0), (2, 3.0, 1.0), (2, 5.0, 2.0), (2, 10.0, 3.0)], ... ("id", "v", "w")) >>> df.groupby("id").agg(weighted_mean(v=df["v"], w=df["w"])).show() +---+-+ | id|weighted_mean(v => v, w => w)| +---+-+ | 1| 1.6667| | 2|7.167| +---+-+ >>> df.groupby("id").agg(weighted_mean(w=df["w"], v=df["v"])).show() +---+-+ | id|weighted_mean(w => w, v => v)| +---+-+ | 1| 1.6667| | 2|7.167| +---+-+ ``` or with window: ```py >>> w = Window.partitionBy("id").orderBy("v").rowsBetween(-2, 1) >>> df.withColumn("wm", weighted_mean(v=df.v, w=df.w).over(w)).show() +---++---+--+ | id| v| w|wm| +---++---+--+ | 1| 1.0|1.0|1.6667| | 1| 2.0|2.0|1.6667| | 2| 3.0|1.0| 4.333| | 2| 5.0|2.0| 7.167| | 2|10.0|3.0| 7.167| +---++---+--+ >>> df.withColumn("wm", weighted_mean_udf(w=df.w, v=df.v).over(w)).show() +---++---+--+ | id| v| w|wm| +---++---+--+ | 1| 1.0|1.0|1.6667| | 1| 2.0|2.0|1.6667| | 2| 3.0|1.0| 4.333| | 2| 5.0|2.0| 7.167| | 2|10.0|3.0| 7.167| +---++---+--+ ``` ### Why are the changes needed? Now that named arguments support was added (https://github.com/apache/spark/pull/41796, https://github.com/apache/spark/pull/42020). Aggregate Pandas UDFs can support it. ### Does this PR introduce _any_ user-facing change? Yes, named arguments will be available for aggregate Pandas UDFs. ### How was this patch tested? Added related tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #42663 from ueshin/issues/SPARK-44952/kwargs. Authored-by: Takuya UESHIN Signed-off-by: Takuya UESHIN --- python/pyspark/sql/pandas/functions.py | 20 ++- .../tests/pandas/test_pandas_udf_grouped_agg.py| 147 - .../sql/tests/pandas/test_pandas_udf_window.py | 173 - python/pyspark/sql/tests/test_udf.py | 15 ++ python/pyspark/sql/tests/test_udtf.py | 15 ++ python/pyspark/worker.py | 25 +-- .../spark/sql/catalyst/analysis/Analyzer.scala | 11 +- .../execution/python/AggregateInPandasExec.scala | 23 ++- .../python/UserDefinedPythonFunction.scala | 3 +- .../python/WindowInPandasEvaluatorFactory.scala| 37 +++-- 10 files changed, 429 insertions(+), 40 deletions(-) diff --git a/python/pyspark/sql/pandas/functions.py b/python/pyspark/sql/pandas/functions.py index ad9fdac9706..652129180df 100644 --- a/python/pyspark/sql/pandas/functions.py +++ b/python/pyspark/sql/pandas/functions.py @@ -57,7 +57,7 @@ def pandas_udf(f=None, returnType=None, functionType=None): Supports Spark Connect. .. versionchanged:: 4.0.0 -Supports keyword-arguments in SCALAR type. +Supports keyword-arguments in SCALAR and GROUPED_AGG type. Parameters -- @@ -267,6 +267,24 @@ def pandas_udf(f=None, returnType=None, functionType=None): | 2|6.0| +---+---+ +This type of Pandas UDF can use keyword arguments: + +
[spark] branch master updated: [SPARK-44918][SQL][PYTHON] Support named arguments in scalar Python/Pandas UDFs
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new ce6b5f3751e [SPARK-44918][SQL][PYTHON] Support named arguments in scalar Python/Pandas UDFs ce6b5f3751e is described below commit ce6b5f3751e9ea5d1cb4b63c8e14235914817766 Author: Takuya UESHIN AuthorDate: Thu Aug 24 10:55:54 2023 -0700 [SPARK-44918][SQL][PYTHON] Support named arguments in scalar Python/Pandas UDFs ### What changes were proposed in this pull request? Supports named arguments in scalar Python/Pandas UDF. For example: ```py >>> udf("int") ... def test_udf(a, b): ... return a + 10 * b ... >>> spark.udf.register("test_udf", test_udf) >>> spark.range(2).select(test_udf(b=col("id") * 10, a=col("id"))).show() +-+ |test_udf(b => (id * 10), a => id)| +-+ |0| | 101| +-+ >>> spark.sql("SELECT test_udf(b => id * 10, a => id) FROM range(2)").show() +-+ |test_udf(b => (id * 10), a => id)| +-+ |0| | 101| +-+ ``` or: ```py >>> pandas_udf("int") ... def test_udf(a, b): ... return a + 10 * b ... >>> spark.udf.register("test_udf", test_udf) >>> spark.range(2).select(test_udf(b=col("id") * 10, a=col("id"))).show() +-+ |test_udf(b => (id * 10), a => id)| +-+ |0| | 101| +-+ >>> spark.sql("SELECT test_udf(b => id * 10, a => id) FROM range(2)").show() +-+ |test_udf(b => (id * 10), a => id)| +-+ |0| | 101| +-+ ``` ### Why are the changes needed? Now that named arguments support was added (https://github.com/apache/spark/pull/41796, https://github.com/apache/spark/pull/42020). Scalar Python/Pandas UDFs can support it. ### Does this PR introduce _any_ user-facing change? Yes, named arguments will be available for scalar Python/Pandas UDFs. ### How was this patch tested? Added related tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #42617 from ueshin/issues/SPARK-44918/kwargs. Authored-by: Takuya UESHIN Signed-off-by: Takuya UESHIN --- python/pyspark/sql/connect/udf.py | 24 +++--- python/pyspark/sql/functions.py| 17 python/pyspark/sql/pandas/functions.py | 17 .../sql/tests/pandas/test_pandas_udf_scalar.py | 94 - python/pyspark/sql/tests/test_udf.py | 98 +- python/pyspark/sql/tests/test_udtf.py | 4 +- python/pyspark/sql/udf.py | 24 -- python/pyspark/worker.py | 89 ++-- .../ApplyInPandasWithStatePythonRunner.scala | 4 + .../sql/execution/python/ArrowEvalPythonExec.scala | 9 +- .../execution/python/ArrowEvalPythonUDTFExec.scala | 2 +- .../sql/execution/python/ArrowPythonRunner.scala | 51 ++- .../execution/python/ArrowPythonUDTFRunner.scala | 11 +-- .../sql/execution/python/BatchEvalPythonExec.scala | 7 +- .../execution/python/BatchEvalPythonUDTFExec.scala | 20 ++--- .../python/EvalPythonEvaluatorFactory.scala| 25 -- .../sql/execution/python/EvalPythonExec.scala | 10 +++ .../sql/execution/python/EvalPythonUDTFExec.scala | 17 +--- .../sql/execution/python/PythonArrowInput.scala| 11 +-- .../sql/execution/python/PythonUDFRunner.scala | 85 +-- .../python/UserDefinedPythonFunction.scala | 15 +++- 21 files changed, 497 insertions(+), 137 deletions(-) diff --git a/python/pyspark/sql/connect/udf.py b/python/pyspark/sql/connect/udf.py index 2636777e5f6..90cea26e56f 100644 --- a/python/pyspark/sql/connect/udf.py +++ b/python/pyspark/sql/connect/udf.py @@ -25,13 +25,15 @@ import sys import functools import warnings from inspect import getfullargspec -from typing import cast, Ca
[spark] branch master updated: [SPARK-44748][SQL] Query execution for the PARTITION BY clause in UDTF TABLE arguments
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 7f3a439fe31 [SPARK-44748][SQL] Query execution for the PARTITION BY clause in UDTF TABLE arguments 7f3a439fe31 is described below commit 7f3a439fe31e487c08864af4fbd31745c9fb289c Author: Daniel Tenedorio AuthorDate: Mon Aug 21 15:47:03 2023 -0700 [SPARK-44748][SQL] Query execution for the PARTITION BY clause in UDTF TABLE arguments ### What changes were proposed in this pull request? This PR implements query execution support for for the PARTITION BY and ORDER BY clauses for UDTF TABLE arguments. * The query planning support was added in [1] and [2] and [3]. After those changes, the planner added a projection to compute the PARTITION BY expressions, plus a repartition operator, plus a sort operator. * In this PR, the Python executor receives the indexes of these expressions within the input table's rows, and compares the values of the projected partitioning expressions between consecutive rows. * When the values change, this marks the boundary between partitions, and so we call the UDTF instance's `terminate` method, then destroy it and create a new one for the next partition. [1] https://github.com/apache/spark/pull/42100 [2] https://github.com/apache/spark/pull/42174 [3] https://github.com/apache/spark/pull/42351 Example: ``` # Make a test UDTF to yield an output row with the same value # consumed from the last input row in the input table or partition. class TestUDTF: def eval(self, row: Row): self._last = row['input'] self._partition_col = row['partition_col'] def terminate(self): yield self._partition_col, self._last func = udtf(TestUDTF, returnType='partition_col: int, last: int') self.spark.udtf.register('test_udtf', func) self.spark.sql(''' WITH t AS ( SELECT id AS partition_col, 1 AS input FROM range(0, 2) UNION ALL SELECT id AS partition_col, 2 AS input FROM range(0, 2) ) SELECT * FROM test_udtf(TABLE(t) PARTITION BY partition_col ORDER BY input) ''').collect() > [Row(partition_col=0, last=2), (partition_col=1, last=2)] ``` ### Why are the changes needed? This brings full end-to-end execution for the PARTITION BY and/or ORDER BY clauses for UDTF TABLE arguments. ### Does this PR introduce _any_ user-facing change? Yes, see above. ### How was this patch tested? This PR adds end-to-end testing in `test_udtf.py`. Closes #42420 from dtenedor/inspect-partition-by. Authored-by: Daniel Tenedorio Signed-off-by: Takuya UESHIN --- python/pyspark/sql/tests/test_udtf.py | 167 + python/pyspark/worker.py | 89 ++- .../spark/sql/catalyst/analysis/Analyzer.scala | 30 +++- .../FunctionTableSubqueryArgumentExpression.scala | 18 --- .../spark/sql/catalyst/expressions/PythonUDF.scala | 12 +- .../execution/python/BatchEvalPythonUDTFExec.scala | 8 + .../sql/execution/python/PythonUDTFSuite.scala | 26 ++-- 7 files changed, 313 insertions(+), 37 deletions(-) diff --git a/python/pyspark/sql/tests/test_udtf.py b/python/pyspark/sql/tests/test_udtf.py index 59a482ad411..9a80f8fd73c 100644 --- a/python/pyspark/sql/tests/test_udtf.py +++ b/python/pyspark/sql/tests/test_udtf.py @@ -1964,6 +1964,173 @@ class BaseUDTFTestsMixin: with self.subTest(query_no=i): assertDataFrameEqual(df, [Row(a=10, b="z")]) +def test_udtf_with_table_argument_and_partition_by(self): +class TestUDTF: +def __init__(self): +self._sum = 0 +self._partition_col = None + +def eval(self, row: Row): +self._sum += row["input"] +if self._partition_col is not None and self._partition_col != row["partition_col"]: +# Make sure that all values of the partitioning column are the same +# for each row consumed by this method for this instance of the class. +raise Exception( +f"self._partition_col was {self._partition_col} but the row " ++ f"value was {row['partition_col']}" +) +self._partition_col = row["partition_col"] + +def terminate(self): +yield self._partition_col, self._sum + +# This is a basic example. +func = udtf(TestUDTF,
[spark] branch branch-3.5 updated: [SPARK-44834][PYTHON][SQL][TESTS] Add SQL query tests for Python UDTFs
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new e473d892bf3 [SPARK-44834][PYTHON][SQL][TESTS] Add SQL query tests for Python UDTFs e473d892bf3 is described below commit e473d892bf3c9b5d4ff6bdb192553b44b2277279 Author: allisonwang-db AuthorDate: Thu Aug 17 10:38:01 2023 -0700 [SPARK-44834][PYTHON][SQL][TESTS] Add SQL query tests for Python UDTFs ### What changes were proposed in this pull request? This PR adds a new sql query test suite for running Python UDTFs in SQL. You can trigger the test using ``` SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/testOnly *SQLQueryTestSuite -- -z udtf/udtf.sql" ``` ### Why are the changes needed? To add more test cases for Python UDTFs. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added new golden file tests. Closes #42517 from allisonwang-db/spark-44834-udtf-sql-test. Authored-by: allisonwang-db Signed-off-by: Takuya UESHIN (cherry picked from commit be04ac1ace91f6da34b08a1510e41d3ab6f0377b) Signed-off-by: Takuya UESHIN --- .../sql-tests/analyzer-results/udtf/udtf.sql.out | 96 ++ .../test/resources/sql-tests/inputs/udtf/udtf.sql | 18 .../resources/sql-tests/results/udtf/udtf.sql.out | 85 +++ .../apache/spark/sql/IntegratedUDFTestUtils.scala | 40 + .../org/apache/spark/sql/SQLQueryTestSuite.scala | 28 +++ .../thriftserver/ThriftServerQueryTestSuite.scala | 2 + 6 files changed, 269 insertions(+) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udtf/udtf.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udtf/udtf.sql.out new file mode 100644 index 000..acf96794378 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udtf/udtf.sql.out @@ -0,0 +1,96 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW t1 AS VALUES (0, 1), (1, 2) t(c1, c2) +-- !query analysis +CreateViewCommand `t1`, VALUES (0, 1), (1, 2) t(c1, c2), false, true, LocalTempView, true + +- SubqueryAlias t + +- LocalRelation [c1#x, c2#x] + + +-- !query +SELECT * FROM udtf(1, 2) +-- !query analysis +Project [x#x, y#x] ++- Generate TestUDTF(1, 2)#x, false, [x#x, y#x] + +- OneRowRelation + + +-- !query +SELECT * FROM udtf(-1, 0) +-- !query analysis +Project [x#x, y#x] ++- Generate TestUDTF(-1, 0)#x, false, [x#x, y#x] + +- OneRowRelation + + +-- !query +SELECT * FROM udtf(0, -1) +-- !query analysis +Project [x#x, y#x] ++- Generate TestUDTF(0, -1)#x, false, [x#x, y#x] + +- OneRowRelation + + +-- !query +SELECT * FROM udtf(0, 0) +-- !query analysis +Project [x#x, y#x] ++- Generate TestUDTF(0, 0)#x, false, [x#x, y#x] + +- OneRowRelation + + +-- !query +SELECT a, b FROM udtf(1, 2) t(a, b) +-- !query analysis +Project [a#x, b#x] ++- SubqueryAlias t + +- Project [x#x AS a#x, y#x AS b#x] + +- Generate TestUDTF(1, 2)#x, false, [x#x, y#x] + +- OneRowRelation + + +-- !query +SELECT * FROM t1, LATERAL udtf(c1, c2) +-- !query analysis +Project [c1#x, c2#x, x#x, y#x] ++- LateralJoin lateral-subquery#x [c1#x && c2#x], Inner + : +- Generate TestUDTF(outer(c1#x), outer(c2#x))#x, false, [x#x, y#x] + : +- OneRowRelation + +- SubqueryAlias t1 + +- View (`t1`, [c1#x,c2#x]) + +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] ++- SubqueryAlias t + +- LocalRelation [c1#x, c2#x] + + +-- !query +SELECT * FROM t1 LEFT JOIN LATERAL udtf(c1, c2) +-- !query analysis +Project [c1#x, c2#x, x#x, y#x] ++- LateralJoin lateral-subquery#x [c1#x && c2#x], LeftOuter + : +- Generate TestUDTF(outer(c1#x), outer(c2#x))#x, false, [x#x, y#x] + : +- OneRowRelation + +- SubqueryAlias t1 + +- View (`t1`, [c1#x,c2#x]) + +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] ++- SubqueryAlias t + +- LocalRelation [c1#x, c2#x] + + +-- !query +SELECT * FROM udtf(1, 2) t(c1, c2), LATERAL udtf(c1, c2) +-- !query analysis +Project [c1#x, c2#x, x#x, y#x] ++- LateralJoin lateral-subquery#x [c1#x && c2#x], Inner + : +- Generate TestUDTF(outer(c1#x), outer(c2#x))#x, false, [x#x, y#x] + : +- OneRowRelation + +- SubqueryAlias t + +- Project [x#x AS c1#x, y#x AS c2#x] + +- Generate TestUDTF(1, 2)#x, false, [x#x, y#x] ++- OneRowRelation + + +-- !query +SELECT * FROM udtf(cast(rand(0) AS int) + 1, 1) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] diff --git a/sql/core/src/test/resources/sql-tests/inputs/udtf/udtf.sql b/sql/core/src/test/resources/sql-tests/inputs/udtf/u
[spark] branch master updated (047b2247879 -> be04ac1ace9)
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 047b2247879 [SPARK-44849] Expose SparkConnectExecutionManager.listActiveExecutions add be04ac1ace9 [SPARK-44834][PYTHON][SQL][TESTS] Add SQL query tests for Python UDTFs No new revisions were added by this update. Summary of changes: .../sql-tests/analyzer-results/udtf/udtf.sql.out | 96 ++ .../test/resources/sql-tests/inputs/udtf/udtf.sql | 18 .../resources/sql-tests/results/udtf/udtf.sql.out | 85 +++ .../apache/spark/sql/IntegratedUDFTestUtils.scala | 40 + .../org/apache/spark/sql/SQLQueryTestSuite.scala | 28 +++ .../thriftserver/ThriftServerQueryTestSuite.scala | 2 + 6 files changed, 269 insertions(+) create mode 100644 sql/core/src/test/resources/sql-tests/analyzer-results/udtf/udtf.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/inputs/udtf/udtf.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/udtf/udtf.sql.out - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-44836][PYTHON] Refactor Arrow Python UDTF
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 06959e2b812 [SPARK-44836][PYTHON] Refactor Arrow Python UDTF 06959e2b812 is described below commit 06959e2b812b176fce67df7270675fd92baf5fed Author: Takuya UESHIN AuthorDate: Wed Aug 16 20:00:26 2023 -0700 [SPARK-44836][PYTHON] Refactor Arrow Python UDTF ### What changes were proposed in this pull request? Refactors Arrow Python UDTF. ### Why are the changes needed? Arrow Python UDTF is not need to be redefined when creating it. It can be handled in `worker.py`. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? The existing tests. Closes #42520 from ueshin/issues/SPARK-44836/refactor_arrow_udtf. Authored-by: Takuya UESHIN Signed-off-by: Takuya UESHIN --- python/pyspark/sql/connect/udtf.py | 40 ++--- python/pyspark/sql/udtf.py | 114 ++--- python/pyspark/worker.py | 43 -- 3 files changed, 71 insertions(+), 126 deletions(-) diff --git a/python/pyspark/sql/connect/udtf.py b/python/pyspark/sql/connect/udtf.py index ce37832854c..d5ce9180358 100644 --- a/python/pyspark/sql/connect/udtf.py +++ b/python/pyspark/sql/connect/udtf.py @@ -83,34 +83,26 @@ def _create_py_udtf( else: raise e -# Create a regular Python UDTF and check for invalid handler class. -regular_udtf = _create_udtf(cls, returnType, name, PythonEvalType.SQL_TABLE_UDF, deterministic) +eval_type: int = PythonEvalType.SQL_TABLE_UDF -if not arrow_enabled: -return regular_udtf - -from pyspark.sql.pandas.utils import ( -require_minimum_pandas_version, -require_minimum_pyarrow_version, -) - -try: -require_minimum_pandas_version() -require_minimum_pyarrow_version() -except ImportError as e: -warnings.warn( -f"Arrow optimization for Python UDTFs cannot be enabled: {str(e)}. " -f"Falling back to using regular Python UDTFs.", -UserWarning, +if arrow_enabled: +from pyspark.sql.pandas.utils import ( +require_minimum_pandas_version, +require_minimum_pyarrow_version, ) -return regular_udtf -from pyspark.sql.udtf import _vectorize_udtf +try: +require_minimum_pandas_version() +require_minimum_pyarrow_version() +eval_type = PythonEvalType.SQL_ARROW_TABLE_UDF +except ImportError as e: +warnings.warn( +f"Arrow optimization for Python UDTFs cannot be enabled: {str(e)}. " +f"Falling back to using regular Python UDTFs.", +UserWarning, +) -vectorized_udtf = _vectorize_udtf(cls) -return _create_udtf( -vectorized_udtf, returnType, name, PythonEvalType.SQL_ARROW_TABLE_UDF, deterministic -) +return _create_udtf(cls, returnType, name, eval_type, deterministic) class UserDefinedTableFunction: diff --git a/python/pyspark/sql/udtf.py b/python/pyspark/sql/udtf.py index 1ca87aae758..fa0bf548bd0 100644 --- a/python/pyspark/sql/udtf.py +++ b/python/pyspark/sql/udtf.py @@ -19,11 +19,10 @@ User-defined table function related classes and functions """ import pickle from dataclasses import dataclass -from functools import wraps import inspect import sys import warnings -from typing import Any, Iterable, Iterator, Type, TYPE_CHECKING, Optional, Union, Callable +from typing import Any, Type, TYPE_CHECKING, Optional, Union from py4j.java_gateway import JavaObject @@ -112,107 +111,30 @@ def _create_py_udtf( if isinstance(value, str) and value.lower() == "true": arrow_enabled = True -# Create a regular Python UDTF and check for invalid handler class. -regular_udtf = _create_udtf(cls, returnType, name, PythonEvalType.SQL_TABLE_UDF, deterministic) - -if not arrow_enabled: -return regular_udtf - -# Return the regular UDTF if the required dependencies are not satisfied. -try: -require_minimum_pandas_version() -require_minimum_pyarrow_version() -except ImportError as e: -warnings.warn( -f"Arrow optimization for Python UDTFs cannot be enabled: {str(e)}. " -f"Falling back to using regular Python UDTFs.", -UserWarning, -) -return regular_udtf +eval_type: int = PythonEvalType.SQL_TABLE_UDF + +if arrow_enabled: +# Return the regular UDTF if the required dependencies are not satisfied. +try: +require_minimum_p
[spark] branch branch-3.5 updated: [SPARK-44836][PYTHON][3.5] Refactor Arrow Python UDTF
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new 08bf227a4e5 [SPARK-44836][PYTHON][3.5] Refactor Arrow Python UDTF 08bf227a4e5 is described below commit 08bf227a4e5a2b266d91d375e5ed3fd57f9e397e Author: Takuya UESHIN AuthorDate: Wed Aug 16 18:47:19 2023 -0700 [SPARK-44836][PYTHON][3.5] Refactor Arrow Python UDTF ### What changes were proposed in this pull request? This is a backport of https://github.com/apache/spark/pull/42520. Refactors Arrow Python UDTF. ### Why are the changes needed? Arrow Python UDTF is not need to be redefined when creating it. It can be handled in `worker.py`. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? The existing tests. Closes #42522 from ueshin/issues/SPARK-44836/3.5/refactor_arrow_udtf. Authored-by: Takuya UESHIN Signed-off-by: Takuya UESHIN --- python/pyspark/sql/connect/udtf.py | 40 ++- python/pyspark/sql/udtf.py | 99 +++--- python/pyspark/worker.py | 36 -- 3 files changed, 66 insertions(+), 109 deletions(-) diff --git a/python/pyspark/sql/connect/udtf.py b/python/pyspark/sql/connect/udtf.py index 850ffe2b9b4..1e77e564ebc 100644 --- a/python/pyspark/sql/connect/udtf.py +++ b/python/pyspark/sql/connect/udtf.py @@ -83,34 +83,26 @@ def _create_py_udtf( else: raise e -# Create a regular Python UDTF and check for invalid handler class. -regular_udtf = _create_udtf(cls, returnType, name, PythonEvalType.SQL_TABLE_UDF, deterministic) +eval_type: int = PythonEvalType.SQL_TABLE_UDF -if not arrow_enabled: -return regular_udtf - -from pyspark.sql.pandas.utils import ( -require_minimum_pandas_version, -require_minimum_pyarrow_version, -) - -try: -require_minimum_pandas_version() -require_minimum_pyarrow_version() -except ImportError as e: -warnings.warn( -f"Arrow optimization for Python UDTFs cannot be enabled: {str(e)}. " -f"Falling back to using regular Python UDTFs.", -UserWarning, +if arrow_enabled: +from pyspark.sql.pandas.utils import ( +require_minimum_pandas_version, +require_minimum_pyarrow_version, ) -return regular_udtf -from pyspark.sql.udtf import _vectorize_udtf +try: +require_minimum_pandas_version() +require_minimum_pyarrow_version() +eval_type = PythonEvalType.SQL_ARROW_TABLE_UDF +except ImportError as e: +warnings.warn( +f"Arrow optimization for Python UDTFs cannot be enabled: {str(e)}. " +f"Falling back to using regular Python UDTFs.", +UserWarning, +) -vectorized_udtf = _vectorize_udtf(cls) -return _create_udtf( -vectorized_udtf, returnType, name, PythonEvalType.SQL_ARROW_TABLE_UDF, deterministic -) +return _create_udtf(cls, returnType, name, eval_type, deterministic) class UserDefinedTableFunction: diff --git a/python/pyspark/sql/udtf.py b/python/pyspark/sql/udtf.py index bf85b55fea3..063fcefd0bf 100644 --- a/python/pyspark/sql/udtf.py +++ b/python/pyspark/sql/udtf.py @@ -20,8 +20,7 @@ User-defined table function related classes and functions import pickle import sys import warnings -from functools import wraps -from typing import Any, Iterable, Iterator, Type, TYPE_CHECKING, Optional, Union, Callable +from typing import Any, Type, TYPE_CHECKING, Optional, Union from py4j.java_gateway import JavaObject @@ -76,92 +75,30 @@ def _create_py_udtf( if isinstance(value, str) and value.lower() == "true": arrow_enabled = True -# Create a regular Python UDTF and check for invalid handler class. -regular_udtf = _create_udtf(cls, returnType, name, PythonEvalType.SQL_TABLE_UDF, deterministic) - -if not arrow_enabled: -return regular_udtf - -# Return the regular UDTF if the required dependencies are not satisfied. -try: -require_minimum_pandas_version() -require_minimum_pyarrow_version() -except ImportError as e: -warnings.warn( -f"Arrow optimization for Python UDTFs cannot be enabled: {str(e)}. " -f"Falling back to using regular Python UDTFs.", -UserWarning, -) -return regular_udtf +eval_type: int = PythonEvalType.SQL_TABLE_UDF + +if arrow_enabled: +# Return the regular UDTF if the required dependencies are not
[spark] branch master updated (b9f11143d05 -> ac567b36db9)
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from b9f11143d05 [SPARK-42664][CONNECT] Support `bloomFilter` function for `DataFrameStatFunctions` add ac567b36db9 [SPARK-44705][FOLLOWUP] Fix Deprecation Version of ContetAwareIterator No new revisions were added by this update. Summary of changes: core/src/main/scala/org/apache/spark/ContextAwareIterator.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated (1d562904e4e -> f7002fb25ca)
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 1d562904e4e [SPARK-44795][CONNECT] CodeGenerator Cache should be classloader specific add f7002fb25ca [SPARK-44749][PYTHON][FOLLOWUP][TESTS] Add more tests for named arguments in Python UDTF No new revisions were added by this update. Summary of changes: python/pyspark/sql/functions.py | 2 +- python/pyspark/sql/tests/test_udtf.py | 88 +++ 2 files changed, 80 insertions(+), 10 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-44749][SQL][PYTHON] Support named arguments in Python UDTF
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new d4629563492 [SPARK-44749][SQL][PYTHON] Support named arguments in Python UDTF d4629563492 is described below commit d4629563492ec3090b5bd5b924790507c42f4e86 Author: Takuya UESHIN AuthorDate: Mon Aug 14 08:57:55 2023 -0700 [SPARK-44749][SQL][PYTHON] Support named arguments in Python UDTF ### What changes were proposed in this pull request? Supports named arguments in Python UDTF. For example: ```py >>> udtf(returnType="a: int") ... class TestUDTF: ... def eval(self, a, b): ... yield a, ... >>> spark.udtf.register("test_udtf", TestUDTF) >>> TestUDTF(a=lit(10), b=lit("x")).show() +---+ | a| +---+ | 10| +---+ >>> TestUDTF(b=lit("x"), a=lit(10)).show() +---+ | a| +---+ | 10| +---+ >>> spark.sql("SELECT * FROM test_udtf(a=>10, b=>'x')").show() +---+ | a| +---+ | 10| +---+ >>> spark.sql("SELECT * FROM test_udtf(b=>'x', a=>10)").show() +---+ | a| +---+ | 10| +---+ ``` or: ```py >>> udtf ... class TestUDTF: ... staticmethod ... def analyze(**kwargs: AnalyzeArgument) -> AnalyzeResult: ... return AnalyzeResult( ... StructType( ... [StructField(key, arg.data_type) for key, arg in sorted(kwargs.items())] ... ) ... ) ... def eval(self, **kwargs): ... yield tuple(value for _, value in sorted(kwargs.items())) ... >>> spark.udtf.register("test_udtf", TestUDTF) >>> spark.sql("SELECT * FROM test_udtf(a=>10, b=>'x', x=>100.0)").show() +---+---+-+ | a| b|x| +---+---+-+ | 10| x|100.0| +---+---+-+ >>> spark.sql("SELECT * FROM test_udtf(x=>10, a=>'x', z=>100.0)").show() +---+---+-+ | a| x|z| +---+---+-+ | x| 10|100.0| +---+---+-+ ``` ### Why are the changes needed? Now that named arguments are supported (https://github.com/apache/spark/pull/41796, https://github.com/apache/spark/pull/42020). It should be supported in Python UDTF. ### Does this PR introduce _any_ user-facing change? Yes, named arguments will be available for Python UDTF. ### How was this patch tested? Added related tests. Closes #42422 from ueshin/issues/SPARK-44749/kwargs. Authored-by: Takuya UESHIN Signed-off-by: Takuya UESHIN --- .../main/protobuf/spark/connect/expressions.proto | 9 ++ .../sql/connect/planner/SparkConnectPlanner.scala | 7 ++ python/pyspark/sql/column.py | 14 +++ python/pyspark/sql/connect/expressions.py | 20 .../pyspark/sql/connect/proto/expressions_pb2.py | 122 +++-- .../pyspark/sql/connect/proto/expressions_pb2.pyi | 34 ++ python/pyspark/sql/connect/udtf.py | 18 +-- python/pyspark/sql/functions.py| 38 +++ python/pyspark/sql/tests/test_udtf.py | 88 +++ python/pyspark/sql/udtf.py | 36 -- python/pyspark/sql/worker/analyze_udtf.py | 20 +++- python/pyspark/worker.py | 29 - .../plans/logical/FunctionBuilderBase.scala| 67 +++ .../execution/python/ArrowEvalPythonUDTFExec.scala | 5 +- .../execution/python/ArrowPythonUDTFRunner.scala | 8 +- .../execution/python/BatchEvalPythonUDTFExec.scala | 30 +++-- .../sql/execution/python/EvalPythonUDTFExec.scala | 33 -- .../python/UserDefinedPythonFunction.scala | 42 +-- 18 files changed, 472 insertions(+), 148 deletions(-) diff --git a/connector/connect/common/src/main/protobuf/spark/connect/expressions.proto b/connector/connect/common/src/main/protobuf/spark/connect/expressions.proto index 557b9db9123..b222f663cd0 100644 --- a/connector/connect/common/src/main/protobuf/spark/connect/expressions.proto +++ b/connector/connect/common/src/main/protobuf/spark/connect/expressions.proto @@ -47,6 +47,7 @@ message Expression { UnresolvedNamedLambdaVariable unresolved_named_lambda_variable = 14; CommonInlineUserDefinedFunction common_inline_user_defined_function = 15; CallFunction call_function = 16; +NamedArgumentExpression named_argument_expression = 17; // This field is used to mark extensions
[spark] branch master updated: [SPARK-44503][SQL] Project any PARTITION BY expressions not already returned from Python UDTF TABLE arguments
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 87298db43d9 [SPARK-44503][SQL] Project any PARTITION BY expressions not already returned from Python UDTF TABLE arguments 87298db43d9 is described below commit 87298db43d9a33fa3a3986f274442a17aad74dc3 Author: Daniel Tenedorio AuthorDate: Wed Aug 9 10:27:07 2023 -0700 [SPARK-44503][SQL] Project any PARTITION BY expressions not already returned from Python UDTF TABLE arguments ### What changes were proposed in this pull request? This PR adds a projection when any Python UDTF TABLE argument contains PARTITION BY expressions that are not simple attributes that are already present in the output of the relation. For example: ``` CREATE TABLE t(d DATE, y INT) USING PARQUET; INSERT INTO t VALUES ... SELECT * FROM UDTF(TABLE(t) PARTITION BY EXTRACT(YEAR FROM d) ORDER BY y ASC); ``` This will generate a plan like: ``` +- Sort (y ASC) +- RepartitionByExpressions (partition_by_0) +- Project (t.d, t.y, EXTRACT(YEAR FROM t.d) AS partition_by_0) +- LogicalRelation "t" ``` ### Why are the changes needed? We project the PARTITION BY expressions so that their resulting values appear in attributes that the Python UDTF interpreter can simply inspect in order to know when the partition boundaries have changed. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? This PR adds unit test coverage. Closes #42351 from dtenedor/partition-by-execution. Authored-by: Daniel Tenedorio Signed-off-by: Takuya UESHIN --- .../FunctionTableSubqueryArgumentExpression.scala | 77 +++-- .../sql/execution/python/PythonUDTFSuite.scala | 127 +++-- 2 files changed, 184 insertions(+), 20 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala index e7a4888125d..daa0751eedf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FunctionTableSubqueryArgumentExpression.scala @@ -104,23 +104,80 @@ case class FunctionTableSubqueryArgumentExpression( // the query plan. var subquery = plan if (partitionByExpressions.nonEmpty) { - subquery = RepartitionByExpression( -partitionExpressions = partitionByExpressions, -child = subquery, -optNumPartitions = None) + // Add a projection to project each of the partitioning expressions that it is not a simple + // attribute that is already present in the plan output. Then add a sort operation by the + // partition keys (plus any explicit ORDER BY items) since after the hash-based shuffle + // operation, the rows from several partitions may arrive interleaved. In this way, the Python + // UDTF evaluator is able to inspect the values of the partitioning expressions for adjacent + // rows in order to determine when each partition ends and the next one begins. + subquery = Project( +projectList = subquery.output ++ extraProjectedPartitioningExpressions, +child = subquery) + val partitioningAttributes = partitioningExpressionIndexes.map(i => subquery.output(i)) + subquery = Sort( +order = partitioningAttributes.map(e => SortOrder(e, Ascending)) ++ orderByExpressions, +global = false, +child = RepartitionByExpression( + partitionExpressions = partitioningAttributes, + optNumPartitions = None, + child = subquery)) } if (withSinglePartition) { subquery = Repartition( numPartitions = 1, shuffle = true, child = subquery) -} -if (orderByExpressions.nonEmpty) { - subquery = Sort( -order = orderByExpressions, -global = false, -child = subquery) + if (orderByExpressions.nonEmpty) { +subquery = Sort( + order = orderByExpressions, + global = false, + child = subquery) + } } Project(Seq(Alias(CreateStruct(subquery.output), "c")()), subquery) } + + /** + * These are the indexes of the PARTITION BY expressions within the concatenation of the child's + * output attributes and the [[extraProjectedPartitioningExpressions]]. We send these indexes to + * the Python UDTF evaluator so it knows which expressions to compare on adjacent rows to kn
[spark] branch branch-3.5 updated: [SPARK-44561][PYTHON] Fix AssertionError when converting UDTF output to a complex type
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new ea6b41cb398 [SPARK-44561][PYTHON] Fix AssertionError when converting UDTF output to a complex type ea6b41cb398 is described below commit ea6b41cb3989996e45102b1930b1498324761093 Author: Takuya UESHIN AuthorDate: Mon Aug 7 11:48:24 2023 -0700 [SPARK-44561][PYTHON] Fix AssertionError when converting UDTF output to a complex type ### What changes were proposed in this pull request? Fixes AssertionError when converting UDTF output to a complex type by ignore assertions in `_create_converter_from_pandas` to make Arrow raise an error. ### Why are the changes needed? There is an assertion in `_create_converter_from_pandas`, but it should not be applied for Python UDTF case. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added/modified the related tests. Closes #42310 from ueshin/issues/SPARK-44561/udtf_complex_types. Authored-by: Takuya UESHIN Signed-off-by: Takuya UESHIN (cherry picked from commit f1a161cb39504bd625ea7fa50d2cc72a1a2a59e9) Signed-off-by: Takuya UESHIN --- python/pyspark/sql/pandas/serializers.py | 5 +- python/pyspark/sql/pandas/types.py | 108 ++--- .../pyspark/sql/tests/connect/test_parity_udtf.py | 3 + python/pyspark/sql/tests/test_udtf.py | 247 +++-- 4 files changed, 314 insertions(+), 49 deletions(-) diff --git a/python/pyspark/sql/pandas/serializers.py b/python/pyspark/sql/pandas/serializers.py index f3037c8b39c..d1a3babb1fd 100644 --- a/python/pyspark/sql/pandas/serializers.py +++ b/python/pyspark/sql/pandas/serializers.py @@ -571,7 +571,10 @@ class ArrowStreamPandasUDTFSerializer(ArrowStreamPandasUDFSerializer): dt = spark_type or from_arrow_type(arrow_type, prefer_timestamp_ntz=True) # TODO(SPARK-43579): cache the converter for reuse conv = _create_converter_from_pandas( -dt, timezone=self._timezone, error_on_duplicated_field_names=False +dt, +timezone=self._timezone, +error_on_duplicated_field_names=False, +ignore_unexpected_complex_type_values=True, ) series = conv(series) diff --git a/python/pyspark/sql/pandas/types.py b/python/pyspark/sql/pandas/types.py index 53362047604..b02a003e632 100644 --- a/python/pyspark/sql/pandas/types.py +++ b/python/pyspark/sql/pandas/types.py @@ -21,7 +21,7 @@ pandas instances during the type conversion. """ import datetime import itertools -from typing import Any, Callable, List, Optional, Union, TYPE_CHECKING +from typing import Any, Callable, Iterable, List, Optional, Union, TYPE_CHECKING from pyspark.sql.types import ( cast, @@ -750,6 +750,7 @@ def _create_converter_from_pandas( *, timezone: Optional[str], error_on_duplicated_field_names: bool = True, +ignore_unexpected_complex_type_values: bool = False, ) -> Callable[["pd.Series"], "pd.Series"]: """ Create a converter of pandas Series to create Spark DataFrame with Arrow optimization. @@ -763,6 +764,17 @@ def _create_converter_from_pandas( error_on_duplicated_field_names : bool, optional Whether raise an exception when there are duplicated field names. (default ``True``) +ignore_unexpected_complex_type_values : bool, optional +Whether ignore the case where unexpected values are given for complex types. +If ``False``, each complex type expects: + +* array type: :class:`Iterable` +* map type: :class:`dict` +* struct type: :class:`dict` or :class:`tuple` + +and raise an AssertionError when the given value is not the expected type. +If ``True``, just ignore and return the give value. +(default ``False``) Returns --- @@ -781,15 +793,26 @@ def _create_converter_from_pandas( def _converter(dt: DataType) -> Optional[Callable[[Any], Any]]: if isinstance(dt, ArrayType): -_element_conv = _converter(dt.elementType) -if _element_conv is None: -return None +_element_conv = _converter(dt.elementType) or (lambda x: x) -def convert_array(value: Any) -> Any: -if value is None: -return None -else: -return [_element_conv(v) for v in value] # type: ignore[misc] +if ignore_unexpected_complex_type_values: + +def convert_array(value: Any) -> Any: +if value is None: +
[spark] branch master updated: [SPARK-44561][PYTHON] Fix AssertionError when converting UDTF output to a complex type
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new f1a161cb395 [SPARK-44561][PYTHON] Fix AssertionError when converting UDTF output to a complex type f1a161cb395 is described below commit f1a161cb39504bd625ea7fa50d2cc72a1a2a59e9 Author: Takuya UESHIN AuthorDate: Mon Aug 7 11:48:24 2023 -0700 [SPARK-44561][PYTHON] Fix AssertionError when converting UDTF output to a complex type ### What changes were proposed in this pull request? Fixes AssertionError when converting UDTF output to a complex type by ignore assertions in `_create_converter_from_pandas` to make Arrow raise an error. ### Why are the changes needed? There is an assertion in `_create_converter_from_pandas`, but it should not be applied for Python UDTF case. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added/modified the related tests. Closes #42310 from ueshin/issues/SPARK-44561/udtf_complex_types. Authored-by: Takuya UESHIN Signed-off-by: Takuya UESHIN --- python/pyspark/sql/pandas/serializers.py | 5 +- python/pyspark/sql/pandas/types.py | 108 ++--- .../pyspark/sql/tests/connect/test_parity_udtf.py | 3 + python/pyspark/sql/tests/test_udtf.py | 247 +++-- 4 files changed, 314 insertions(+), 49 deletions(-) diff --git a/python/pyspark/sql/pandas/serializers.py b/python/pyspark/sql/pandas/serializers.py index f3037c8b39c..d1a3babb1fd 100644 --- a/python/pyspark/sql/pandas/serializers.py +++ b/python/pyspark/sql/pandas/serializers.py @@ -571,7 +571,10 @@ class ArrowStreamPandasUDTFSerializer(ArrowStreamPandasUDFSerializer): dt = spark_type or from_arrow_type(arrow_type, prefer_timestamp_ntz=True) # TODO(SPARK-43579): cache the converter for reuse conv = _create_converter_from_pandas( -dt, timezone=self._timezone, error_on_duplicated_field_names=False +dt, +timezone=self._timezone, +error_on_duplicated_field_names=False, +ignore_unexpected_complex_type_values=True, ) series = conv(series) diff --git a/python/pyspark/sql/pandas/types.py b/python/pyspark/sql/pandas/types.py index 53362047604..b02a003e632 100644 --- a/python/pyspark/sql/pandas/types.py +++ b/python/pyspark/sql/pandas/types.py @@ -21,7 +21,7 @@ pandas instances during the type conversion. """ import datetime import itertools -from typing import Any, Callable, List, Optional, Union, TYPE_CHECKING +from typing import Any, Callable, Iterable, List, Optional, Union, TYPE_CHECKING from pyspark.sql.types import ( cast, @@ -750,6 +750,7 @@ def _create_converter_from_pandas( *, timezone: Optional[str], error_on_duplicated_field_names: bool = True, +ignore_unexpected_complex_type_values: bool = False, ) -> Callable[["pd.Series"], "pd.Series"]: """ Create a converter of pandas Series to create Spark DataFrame with Arrow optimization. @@ -763,6 +764,17 @@ def _create_converter_from_pandas( error_on_duplicated_field_names : bool, optional Whether raise an exception when there are duplicated field names. (default ``True``) +ignore_unexpected_complex_type_values : bool, optional +Whether ignore the case where unexpected values are given for complex types. +If ``False``, each complex type expects: + +* array type: :class:`Iterable` +* map type: :class:`dict` +* struct type: :class:`dict` or :class:`tuple` + +and raise an AssertionError when the given value is not the expected type. +If ``True``, just ignore and return the give value. +(default ``False``) Returns --- @@ -781,15 +793,26 @@ def _create_converter_from_pandas( def _converter(dt: DataType) -> Optional[Callable[[Any], Any]]: if isinstance(dt, ArrayType): -_element_conv = _converter(dt.elementType) -if _element_conv is None: -return None +_element_conv = _converter(dt.elementType) or (lambda x: x) -def convert_array(value: Any) -> Any: -if value is None: -return None -else: -return [_element_conv(v) for v in value] # type: ignore[misc] +if ignore_unexpected_complex_type_values: + +def convert_array(value: Any) -> Any: +if value is None: +return None +elif isinstance(value, Iterable): +retu
[spark] branch branch-3.5 updated: [SPARK-44433][3.5][PYTHON][CONNECT][SS][FOLLOWUP] Terminate listener process with removeListener and improvements
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new f22b618174b [SPARK-44433][3.5][PYTHON][CONNECT][SS][FOLLOWUP] Terminate listener process with removeListener and improvements f22b618174b is described below commit f22b618174b72e70d9f764379ce1689f2da6470d Author: Wei Liu AuthorDate: Fri Aug 4 19:40:47 2023 -0700 [SPARK-44433][3.5][PYTHON][CONNECT][SS][FOLLOWUP] Terminate listener process with removeListener and improvements ### Master Branch PR: https://github.com/apache/spark/pull/42283 ### What changes were proposed in this pull request? This is a followup to #42116. It addresses the following issues: 1. When `removeListener` is called upon one listener, before the python process is left running, now it also get stopped. 2. When multiple `removeListener` is called on the same listener, in non-connect mode, subsequent calls will be noop. But before this PR, in connect it actually throws an error, which doesn't align with existing behavior, this PR addresses it. 3. Set the socket timeout to be None (\infty) for `foreachBatch_worker` and `listener_worker`, because there could be a long time between each microbatch. If not setting this, the socket will timeout and won't be able to process new data. ``` scala> Streaming query listener worker is starting with url sc://localhost:15002/;user_id=wei.liu and sessionId 886191f0-2b64-4c44-b067-de511f04b42d. Traceback (most recent call last): File "/usr/lib/python3.9/runpy.py", line 197, in _run_module_as_main return _run_code(code, main_globals, None, File "/usr/lib/python3.9/runpy.py", line 87, in _run_code exec(code, run_globals) File "/home/wei.liu/oss-spark/python/lib/pyspark.zip/pyspark/sql/connect/streaming/worker/listener_worker.py", line 95, in File "/home/wei.liu/oss-spark/python/lib/pyspark.zip/pyspark/sql/connect/streaming/worker/listener_worker.py", line 82, in main File "/home/wei.liu/oss-spark/python/lib/pyspark.zip/pyspark/serializers.py", line 557, in loads File "/home/wei.liu/oss-spark/python/lib/pyspark.zip/pyspark/serializers.py", line 594, in read_int File "/usr/lib/python3.9/socket.py", line 704, in readinto return self._sock.recv_into(b) socket.timeout: timed out ``` ### Why are the changes needed? Necessary improvements ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Manual test + unit test Closes #42340 from WweiL/SPARK-44433-listener-followup-3.5. Authored-by: Wei Liu Signed-off-by: Takuya UESHIN --- .../sql/streaming/StreamingQueryListener.scala | 28 .../sql/connect/planner/SparkConnectPlanner.scala | 12 +++-- .../planner/StreamingForeachBatchHelper.scala | 10 +++-- .../planner/StreamingQueryListenerHelper.scala | 21 + .../spark/sql/connect/service/SessionHolder.scala | 19 .../spark/api/python/StreamingPythonRunner.scala | 51 +- .../streaming/worker/foreachBatch_worker.py| 4 +- .../connect/streaming/worker/listener_worker.py| 4 +- .../connect/streaming/test_parity_listener.py | 7 +++ 9 files changed, 89 insertions(+), 67 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala index e2f3be02ad3..404bd1b078b 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala @@ -75,34 +75,6 @@ abstract class StreamingQueryListener extends Serializable { def onQueryTerminated(event: QueryTerminatedEvent): Unit } -/** - * Py4J allows a pure interface so this proxy is required. - */ -private[spark] trait PythonStreamingQueryListener { - import StreamingQueryListener._ - - def onQueryStarted(event: QueryStartedEvent): Unit - - def onQueryProgress(event: QueryProgressEvent): Unit - - def onQueryIdle(event: QueryIdleEvent): Unit - - def onQueryTerminated(event: QueryTerminatedEvent): Unit -} - -private[spark] class PythonStreamingQueryListenerWrapper(listener: PythonStreamingQueryListener) -extends StreamingQueryListener { - import StreamingQueryListener._ - - def onQueryStarted(event: QueryStartedEvent): Unit = listener.onQueryStarted(event) - - def onQueryProgress(event: QueryProgressEvent): Unit = listener.onQueryProgress(event)
[spark] branch master updated: [SPARK-44433][PYTHON][CONNECT][SS][FOLLOWUP] Set back USE_DAEMON after creating streaming python processes
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new cf64008fce7 [SPARK-44433][PYTHON][CONNECT][SS][FOLLOWUP] Set back USE_DAEMON after creating streaming python processes cf64008fce7 is described below commit cf64008fce77b38d1237874b04f5ac124b01b3a8 Author: Wei Liu AuthorDate: Fri Aug 4 17:41:27 2023 -0700 [SPARK-44433][PYTHON][CONNECT][SS][FOLLOWUP] Set back USE_DAEMON after creating streaming python processes ### What changes were proposed in this pull request? Followup of this comment: https://github.com/apache/spark/pull/42283#discussion_r1283804782 Change back the spark conf after creating streaming python process. ### Why are the changes needed? Bug fix ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Config only change Closes #42341 from WweiL/SPARK-44433-followup-USEDAEMON. Authored-by: Wei Liu Signed-off-by: Takuya UESHIN --- .../spark/api/python/StreamingPythonRunner.scala | 18 -- 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/StreamingPythonRunner.scala b/core/src/main/scala/org/apache/spark/api/python/StreamingPythonRunner.scala index f14289f984a..a079743c847 100644 --- a/core/src/main/scala/org/apache/spark/api/python/StreamingPythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/StreamingPythonRunner.scala @@ -66,14 +66,19 @@ private[spark] class StreamingPythonRunner( envVars.put("SPARK_AUTH_SOCKET_TIMEOUT", authSocketTimeout.toString) envVars.put("SPARK_BUFFER_SIZE", bufferSize.toString) -conf.set(PYTHON_USE_DAEMON, false) envVars.put("SPARK_CONNECT_LOCAL_URL", connectUrl) -val (worker, _) = env.createPythonWorker( - pythonExec, workerModule, envVars.asScala.toMap) -pythonWorker = Some(worker) +val prevConf = conf.get(PYTHON_USE_DAEMON) +conf.set(PYTHON_USE_DAEMON, false) +try { + val (worker, _) = env.createPythonWorker( +pythonExec, workerModule, envVars.asScala.toMap) + pythonWorker = Some(worker) +} finally { + conf.set(PYTHON_USE_DAEMON, prevConf) +} -val stream = new BufferedOutputStream(worker.getOutputStream, bufferSize) +val stream = new BufferedOutputStream(pythonWorker.get.getOutputStream, bufferSize) val dataOut = new DataOutputStream(stream) // TODO(SPARK-44461): verify python version @@ -87,7 +92,8 @@ private[spark] class StreamingPythonRunner( dataOut.write(command.toArray) dataOut.flush() -val dataIn = new DataInputStream(new BufferedInputStream(worker.getInputStream, bufferSize)) +val dataIn = new DataInputStream( + new BufferedInputStream(pythonWorker.get.getInputStream, bufferSize)) val resFromPython = dataIn.readInt() logInfo(s"Runner initialization returned $resFromPython") - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch branch-3.5 updated: [SPARK-44663][PYTHON] Disable arrow optimization by default for Python UDTFs
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new 4cba54d8c0e [SPARK-44663][PYTHON] Disable arrow optimization by default for Python UDTFs 4cba54d8c0e is described below commit 4cba54d8c0e113a2587082235518be738c3d4dda Author: allisonwang-db AuthorDate: Fri Aug 4 16:44:01 2023 -0700 [SPARK-44663][PYTHON] Disable arrow optimization by default for Python UDTFs ### What changes were proposed in this pull request? This PR disables arrow optimization by default for Python UDTFs. ### Why are the changes needed? To make Python UDTFs consistent with Python UDFs (arrow optimization is by default disabled). ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New unit tests Closes #42329 from allisonwang-db/spark-44663-disable-arrow. Authored-by: allisonwang-db Signed-off-by: Takuya UESHIN (cherry picked from commit 8b53fed7ef0edaaf948ec67413017e60444230fd) Signed-off-by: Takuya UESHIN --- python/pyspark/sql/connect/udtf.py | 10 +- python/pyspark/sql/tests/test_udtf.py| 16 python/pyspark/sql/udtf.py | 10 +- .../scala/org/apache/spark/sql/internal/SQLConf.scala| 2 +- 4 files changed, 27 insertions(+), 11 deletions(-) diff --git a/python/pyspark/sql/connect/udtf.py b/python/pyspark/sql/connect/udtf.py index 3747e37459e..07e2bad6ec7 100644 --- a/python/pyspark/sql/connect/udtf.py +++ b/python/pyspark/sql/connect/udtf.py @@ -70,11 +70,11 @@ def _create_py_udtf( else: from pyspark.sql.connect.session import _active_spark_session -arrow_enabled = ( - _active_spark_session.conf.get("spark.sql.execution.pythonUDTF.arrow.enabled") == "true" -if _active_spark_session is not None -else True -) +arrow_enabled = False +if _active_spark_session is not None: +value = _active_spark_session.conf.get("spark.sql.execution.pythonUDTF.arrow.enabled") +if isinstance(value, str) and value.lower() == "true": +arrow_enabled = True # Create a regular Python UDTF and check for invalid handler class. regular_udtf = _create_udtf(cls, returnType, name, PythonEvalType.SQL_TABLE_UDF, deterministic) diff --git a/python/pyspark/sql/tests/test_udtf.py b/python/pyspark/sql/tests/test_udtf.py index 4bab77038e0..4a65a9bd2e4 100644 --- a/python/pyspark/sql/tests/test_udtf.py +++ b/python/pyspark/sql/tests/test_udtf.py @@ -1002,6 +1002,22 @@ class UDTFArrowTestsMixin(BaseUDTFTestsMixin): PythonEvalType.SQL_ARROW_TABLE_UDF, ) +def test_udtf_arrow_sql_conf(self): +class TestUDTF: +def eval(self): +yield 1, + +# We do not use `self.sql_conf` here to test the SQL SET command +# instead of using PySpark's `spark.conf.set`. +old_value = self.spark.conf.get("spark.sql.execution.pythonUDTF.arrow.enabled") +self.spark.sql("SET spark.sql.execution.pythonUDTF.arrow.enabled=False") +self.assertEqual(udtf(TestUDTF, returnType="x: int").evalType, PythonEvalType.SQL_TABLE_UDF) +self.spark.sql("SET spark.sql.execution.pythonUDTF.arrow.enabled=True") +self.assertEqual( +udtf(TestUDTF, returnType="x: int").evalType, PythonEvalType.SQL_ARROW_TABLE_UDF +) +self.spark.conf.set("spark.sql.execution.pythonUDTF.arrow.enabled", old_value) + def test_udtf_eval_returning_non_tuple(self): class TestUDTF: def eval(self, a: int): diff --git a/python/pyspark/sql/udtf.py b/python/pyspark/sql/udtf.py index c2830d56db5..7cbf4732ba9 100644 --- a/python/pyspark/sql/udtf.py +++ b/python/pyspark/sql/udtf.py @@ -69,11 +69,11 @@ def _create_py_udtf( from pyspark.sql import SparkSession session = SparkSession._instantiatedSession -arrow_enabled = ( -session.conf.get("spark.sql.execution.pythonUDTF.arrow.enabled") == "true" -if session is not None -else True -) +arrow_enabled = False +if session is not None: +value = session.conf.get("spark.sql.execution.pythonUDTF.arrow.enabled") +if isinstance(value, str) and value.lower() == "true": +arrow_enabled = True # Create a regular Python UDTF and check for invalid handler class. regular_udtf = _create_udtf(cls, returnType, name, PythonEvalType.SQL_TABLE_UDF, deterministic) diff --git a/s
[spark] branch master updated: [SPARK-44663][PYTHON] Disable arrow optimization by default for Python UDTFs
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 8b53fed7ef0 [SPARK-44663][PYTHON] Disable arrow optimization by default for Python UDTFs 8b53fed7ef0 is described below commit 8b53fed7ef0edaaf948ec67413017e60444230fd Author: allisonwang-db AuthorDate: Fri Aug 4 16:44:01 2023 -0700 [SPARK-44663][PYTHON] Disable arrow optimization by default for Python UDTFs ### What changes were proposed in this pull request? This PR disables arrow optimization by default for Python UDTFs. ### Why are the changes needed? To make Python UDTFs consistent with Python UDFs (arrow optimization is by default disabled). ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New unit tests Closes #42329 from allisonwang-db/spark-44663-disable-arrow. Authored-by: allisonwang-db Signed-off-by: Takuya UESHIN --- python/pyspark/sql/connect/udtf.py | 10 +- python/pyspark/sql/tests/test_udtf.py| 16 python/pyspark/sql/udtf.py | 10 +- .../scala/org/apache/spark/sql/internal/SQLConf.scala| 2 +- 4 files changed, 27 insertions(+), 11 deletions(-) diff --git a/python/pyspark/sql/connect/udtf.py b/python/pyspark/sql/connect/udtf.py index 919994401c8..5a95075a655 100644 --- a/python/pyspark/sql/connect/udtf.py +++ b/python/pyspark/sql/connect/udtf.py @@ -70,11 +70,11 @@ def _create_py_udtf( else: from pyspark.sql.connect.session import _active_spark_session -arrow_enabled = ( - _active_spark_session.conf.get("spark.sql.execution.pythonUDTF.arrow.enabled") == "true" -if _active_spark_session is not None -else True -) +arrow_enabled = False +if _active_spark_session is not None: +value = _active_spark_session.conf.get("spark.sql.execution.pythonUDTF.arrow.enabled") +if isinstance(value, str) and value.lower() == "true": +arrow_enabled = True # Create a regular Python UDTF and check for invalid handler class. regular_udtf = _create_udtf(cls, returnType, name, PythonEvalType.SQL_TABLE_UDF, deterministic) diff --git a/python/pyspark/sql/tests/test_udtf.py b/python/pyspark/sql/tests/test_udtf.py index 4d36b537995..9caf267e48d 100644 --- a/python/pyspark/sql/tests/test_udtf.py +++ b/python/pyspark/sql/tests/test_udtf.py @@ -1723,6 +1723,22 @@ class UDTFArrowTestsMixin(BaseUDTFTestsMixin): PythonEvalType.SQL_ARROW_TABLE_UDF, ) +def test_udtf_arrow_sql_conf(self): +class TestUDTF: +def eval(self): +yield 1, + +# We do not use `self.sql_conf` here to test the SQL SET command +# instead of using PySpark's `spark.conf.set`. +old_value = self.spark.conf.get("spark.sql.execution.pythonUDTF.arrow.enabled") +self.spark.sql("SET spark.sql.execution.pythonUDTF.arrow.enabled=False") +self.assertEqual(udtf(TestUDTF, returnType="x: int").evalType, PythonEvalType.SQL_TABLE_UDF) +self.spark.sql("SET spark.sql.execution.pythonUDTF.arrow.enabled=True") +self.assertEqual( +udtf(TestUDTF, returnType="x: int").evalType, PythonEvalType.SQL_ARROW_TABLE_UDF +) +self.spark.conf.set("spark.sql.execution.pythonUDTF.arrow.enabled", old_value) + def test_udtf_eval_returning_non_tuple(self): class TestUDTF: def eval(self, a: int): diff --git a/python/pyspark/sql/udtf.py b/python/pyspark/sql/udtf.py index fea0f74c8f2..027a2646a46 100644 --- a/python/pyspark/sql/udtf.py +++ b/python/pyspark/sql/udtf.py @@ -106,11 +106,11 @@ def _create_py_udtf( from pyspark.sql import SparkSession session = SparkSession._instantiatedSession -arrow_enabled = ( -session.conf.get("spark.sql.execution.pythonUDTF.arrow.enabled") == "true" -if session is not None -else True -) +arrow_enabled = False +if session is not None: +value = session.conf.get("spark.sql.execution.pythonUDTF.arrow.enabled") +if isinstance(value, str) and value.lower() == "true": +arrow_enabled = True # Create a regular Python UDTF and check for invalid handler class. regular_udtf = _create_udtf(cls, returnType, name, PythonEvalType.SQL_TABLE_UDF, deterministic) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/
[spark] branch master updated (780bae92839 -> 62415dc5962)
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 780bae92839 [SPARK-44671][PYTHON][CONNECT] Retry ExecutePlan in case initial request didn't reach server in Python client add 62415dc5962 [SPARK-44644][PYTHON] Improve error messages for Python UDTFs with pickling errors No new revisions were added by this update. Summary of changes: python/pyspark/cloudpickle/cloudpickle_fast.py | 2 +- python/pyspark/errors/error_classes.py | 5 + python/pyspark/sql/connect/plan.py | 15 -- python/pyspark/sql/tests/test_udtf.py | 27 ++ python/pyspark/sql/udtf.py | 25 +++- 5 files changed, 70 insertions(+), 4 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch branch-3.5 updated: [SPARK-44559][PYTHON][3.5] Improve error messages for Python UDTF arrow cast
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new fe3754afcfe [SPARK-44559][PYTHON][3.5] Improve error messages for Python UDTF arrow cast fe3754afcfe is described below commit fe3754afcfe50032a4bf9fabf46dcdea47860626 Author: allisonwang-db AuthorDate: Wed Aug 2 14:28:30 2023 -0700 [SPARK-44559][PYTHON][3.5] Improve error messages for Python UDTF arrow cast ### What changes were proposed in this pull request? This PR cherry-picks https://github.com/apache/spark/commit/5384f4601a4ba8daba76d67e945eaa6fc2b70b2c. It improves error messages when the output of an arrow-optimized Python UDTF cannot be casted to the specified return schema of the UDTF. ### Why are the changes needed? To make Python UDTFs more user-friendly. ### Does this PR introduce _any_ user-facing change? Yes, before this PR, when the output of a UDTF fails to cast to the desired schema, Spark will throw this confusing error message: ```python udtf(returnType="x: int") class TestUDTF: def eval(self): yield [1, 2], TestUDTF().collect() ``` ``` File "pyarrow/array.pxi", line 1044, in pyarrow.lib.Array.from_pandas File "pyarrow/array.pxi", line 316, in pyarrow.lib.array File "pyarrow/array.pxi", line 83, in pyarrow.lib._ndarray_to_array File "pyarrow/error.pxi", line 100, in pyarrow.lib.check_status pyarrow.lib.ArrowInvalid: Could not convert [1, 2] with type list: tried to convert to int32 ``` Now, after this PR, the error message will look like this: `pyspark.errors.exceptions.base.PySparkRuntimeError: [UDTF_ARROW_TYPE_CAST_ERROR] Cannot convert the output value of the column 'x' with type 'object' to the specified return type of the column: 'int32'. Please check if the data types match and try again. ` ### How was this patch tested? New unit tests Closes #42290 from allisonwang-db/spark-44559-3.5. Authored-by: allisonwang-db Signed-off-by: Takuya UESHIN --- python/pyspark/errors/error_classes.py | 5 + python/pyspark/sql/pandas/serializers.py | 69 +++- python/pyspark/sql/tests/test_udtf.py| 259 +++ 3 files changed, 332 insertions(+), 1 deletion(-) diff --git a/python/pyspark/errors/error_classes.py b/python/pyspark/errors/error_classes.py index 554a25952b9..db80705e7d2 100644 --- a/python/pyspark/errors/error_classes.py +++ b/python/pyspark/errors/error_classes.py @@ -713,6 +713,11 @@ ERROR_CLASSES_JSON = """ "Expected values for ``, got ." ] }, + "UDTF_ARROW_TYPE_CAST_ERROR" : { +"message" : [ + "Cannot convert the output value of the column '' with type '' to the specified return type of the column: ''. Please check if the data types match and try again." +] + }, "UDTF_EXEC_ERROR" : { "message" : [ "User defined table function encountered an error in the '' method: " diff --git a/python/pyspark/sql/pandas/serializers.py b/python/pyspark/sql/pandas/serializers.py index 1d326928e23..993bacbed67 100644 --- a/python/pyspark/sql/pandas/serializers.py +++ b/python/pyspark/sql/pandas/serializers.py @@ -19,7 +19,7 @@ Serializers for PyArrow and pandas conversions. See `pyspark.serializers` for more details. """ -from pyspark.errors import PySparkTypeError, PySparkValueError +from pyspark.errors import PySparkRuntimeError, PySparkTypeError, PySparkValueError from pyspark.serializers import Serializer, read_int, write_int, UTF8Deserializer, CPickleSerializer from pyspark.sql.pandas.types import ( from_arrow_type, @@ -538,6 +538,73 @@ class ArrowStreamPandasUDTFSerializer(ArrowStreamPandasUDFSerializer): return pa.RecordBatch.from_arrays(arrs, ["_%d" % i for i in range(len(arrs))]) +def _create_array(self, series, arrow_type, spark_type=None, arrow_cast=False): +""" +Override the `_create_array` method in the superclass to create an Arrow Array +from a given pandas.Series and an arrow type. The difference here is that we always +use arrow cast when creating the arrow array. Also, the error messages are specific +to arrow-optimized Python UDTFs. + +Parameters +-- +series : pandas.Series +A single series +arrow_type : pyarrow.DataType, optional +If None, pyarrow's inferred type will be used +spark_type : DataType, optional +
[spark] branch master updated (e326b9b6565 -> cfc75ec12ae)
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from e326b9b6565 [SPARK-41532][DOCS][FOLLOWUP] Regenerate `sql-error-conditions-connect-error-class.md` add cfc75ec12ae [SPARK-44503][SQL] Add analysis and planning for PARTITION BY and ORDER BY clause after TABLE arguments for TVF calls No new revisions were added by this update. Summary of changes: docs/sql-ref-ansi-compliance.md| 1 + .../spark/sql/catalyst/parser/SqlBaseLexer.g4 | 1 + .../spark/sql/catalyst/parser/SqlBaseParser.g4 | 11 +- .../spark/sql/catalyst/analysis/Analyzer.scala | 12 +- .../FunctionTableSubqueryArgumentExpression.scala | 69 +- .../spark/sql/catalyst/parser/AstBuilder.scala | 32 +++-- .../sql/catalyst/parser/PlanParserSuite.scala | 152 + .../sql-tests/results/ansi/keywords.sql.out| 1 + .../resources/sql-tests/results/keywords.sql.out | 1 + .../sql/execution/python/PythonUDTFSuite.scala | 67 - .../ThriftServerWithSparkContextSuite.scala| 2 +- 11 files changed, 307 insertions(+), 42 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch branch-3.5 updated: [SPARK-44614][PYTHON][CONNECT][3.5] Add missing packages in setup.py
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new 84f17e7dcd5 [SPARK-44614][PYTHON][CONNECT][3.5] Add missing packages in setup.py 84f17e7dcd5 is described below commit 84f17e7dcd5e6fa71a1a35a58d0ea19170e897a5 Author: Takuya UESHIN AuthorDate: Tue Aug 1 08:15:45 2023 -0700 [SPARK-44614][PYTHON][CONNECT][3.5] Add missing packages in setup.py ### What changes were proposed in this pull request? Adds missing packages in `setup.py`. ### Why are the changes needed? The following packages are not listed in `setup.py`. - `pyspark.sql.connect.avro` - `pyspark.sql.connect.client` - `pyspark.sql.connect.streaming` - `pyspark.ml.connect` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? The existing tests. Closes #42257 from ueshin/issues/SPARK-44614/3.5/packages. Authored-by: Takuya UESHIN Signed-off-by: Takuya UESHIN --- python/setup.py | 4 1 file changed, 4 insertions(+) diff --git a/python/setup.py b/python/setup.py index a0297d4f9dc..4d14dfd3cb9 100755 --- a/python/setup.py +++ b/python/setup.py @@ -235,6 +235,7 @@ try: "pyspark", "pyspark.cloudpickle", "pyspark.mllib", +"pyspark.ml.connect", "pyspark.mllib.linalg", "pyspark.mllib.stat", "pyspark.ml", @@ -245,7 +246,10 @@ try: "pyspark.sql", "pyspark.sql.avro", "pyspark.sql.connect", +"pyspark.sql.connect.avro", +"pyspark.sql.connect.client", "pyspark.sql.connect.proto", +"pyspark.sql.connect.streaming", "pyspark.sql.pandas", "pyspark.sql.protobuf", "pyspark.sql.streaming", - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-44614][PYTHON][CONNECT] Add missing packages in setup.py
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new a011d16a24d [SPARK-44614][PYTHON][CONNECT] Add missing packages in setup.py a011d16a24d is described below commit a011d16a24db383496cc64c396f08bc64f34e379 Author: Takuya UESHIN AuthorDate: Tue Aug 1 01:08:23 2023 -0700 [SPARK-44614][PYTHON][CONNECT] Add missing packages in setup.py ### What changes were proposed in this pull request? Adds missing packages in `setup.py`. ### Why are the changes needed? The following packages are not listed in `setup.py`. - `pyspark.sql.connect.avro` - `pyspark.sql.connect.client` - `pyspark.sql.connect.streaming` - `pyspark.ml.connect` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? The existing tests. Closes #42248 from ueshin/issues/SPARK-44614/packages. Authored-by: Takuya UESHIN Signed-off-by: Takuya UESHIN --- python/setup.py | 6 +- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/python/setup.py b/python/setup.py index 2282b2535e7..6642ec5042f 100755 --- a/python/setup.py +++ b/python/setup.py @@ -238,6 +238,7 @@ try: "pyspark.mllib.linalg", "pyspark.mllib.stat", "pyspark.ml", +"pyspark.ml.connect", "pyspark.ml.linalg", "pyspark.ml.param", "pyspark.ml.torch", @@ -245,13 +246,16 @@ try: "pyspark.sql", "pyspark.sql.avro", "pyspark.sql.connect", +"pyspark.sql.connect.avro", +"pyspark.sql.connect.client", "pyspark.sql.connect.proto", +"pyspark.sql.connect.streaming", +"pyspark.sql.connect.streaming.worker", "pyspark.sql.pandas", "pyspark.sql.protobuf", "pyspark.sql.streaming", "pyspark.sql.worker", "pyspark.streaming", -"pyspark.sql.connect.streaming.worker", "pyspark.bin", "pyspark.sbin", "pyspark.jars", - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-42941][SS][CONNECT] Python StreamingQueryListener
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 799ab87ebd2 [SPARK-42941][SS][CONNECT] Python StreamingQueryListener 799ab87ebd2 is described below commit 799ab87ebd23fc09181e15dd9b7bfe5a9e0edf86 Author: Wei Liu AuthorDate: Mon Jul 31 14:36:54 2023 -0700 [SPARK-42941][SS][CONNECT] Python StreamingQueryListener ### What changes were proposed in this pull request? Implement the python streaming query listener and the `addListener` method and `removeListener` method, follow up filed in: SPARK-44516 to actually terminate the query listener process when `removeListener` is called. SPARK-44516 depends on SPARK-44433. ### Why are the changes needed? SS Connect development ### Does this PR introduce _any_ user-facing change? Yes now they can use connect listener ### How was this patch tested? Manual test and added unit test addListener: ``` # Client side: >>> from pyspark.sql.streaming.listener import StreamingQueryListener;from pyspark.sql.streaming.listener import (QueryStartedEvent, QueryProgressEvent, QueryTerminatedEvent, QueryIdleEvent) >>> class MyListener(StreamingQueryListener): ... def onQueryStarted(self, event: QueryStartedEvent) -> None: print("hi, event query id is: " + str(event.id)); df=self.spark.createDataFrame(["10","11","13"], "string").toDF("age"); df.write.saveAsTable("tbllistener1") ... def onQueryProgress(self, event: QueryProgressEvent) -> None: pass ... def onQueryIdle(self, event: QueryIdleEvent) -> None: pass ... def onQueryTerminated(self, event: QueryTerminatedEvent) -> None: pass ... >>> spark.streams.addListener(MyListener()) >>> q = spark.readStream.format("rate").load().writeStream.format("console").start() >>> q.stop() >>> spark.read.table("tbllistener1").collect() [Row(age='13'), Row(age='10'), Row(age='11’)] # Server side: # event_type received from python process is 0 hi, event query id is: dd7ba1c4-6c8f-4369-9c3c-5dede22b8a2f ``` removeListener: ``` # Client side: >>> listener = MyListener(); spark.streams.addListener(listener) >>> spark.streams.removeListener(listener) # Server side: # nothing to print actually, the listener is removed from server side StreamingQueryManager and cache in sessionHolder, but the process still hangs there. Follow up SPARK-44516 filed to stop this process ``` Closes #42116 from WweiL/listener-poc-newest. Lead-authored-by: Wei Liu Co-authored-by: pengzhon-db Signed-off-by: Takuya UESHIN --- .../sql/streaming/StreamingQueryManager.scala | 6 +- .../spark/sql/streaming/StreamingQuerySuite.scala | 2 +- .../src/main/protobuf/spark/connect/commands.proto | 2 + .../sql/connect/planner/SparkConnectPlanner.scala | 30 +-- .../planner/StreamingForeachBatchHelper.scala | 9 +- .../planner/StreamingQueryListenerHelper.scala | 69 +++ .../spark/api/python/StreamingPythonRunner.scala | 5 +- dev/sparktestsupport/modules.py| 1 + python/pyspark/sql/connect/proto/commands_pb2.py | 44 ++-- python/pyspark/sql/connect/proto/commands_pb2.pyi | 37 +++- python/pyspark/sql/connect/streaming/query.py | 31 +-- .../sql/connect/streaming/worker/__init__.py | 18 ++ .../streaming/worker/foreachBatch_worker.py} | 18 +- .../connect/streaming/worker/listener_worker.py} | 53 +++-- python/pyspark/sql/streaming/listener.py | 29 ++- python/pyspark/sql/streaming/query.py | 12 ++ .../connect/streaming/test_parity_listener.py | 90 .../sql/tests/streaming/test_streaming_listener.py | 228 +++-- python/setup.py| 1 + 19 files changed, 481 insertions(+), 204 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala index 91744460440..d16638e5945 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala @@ -156,7 +156,8 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo executeManagerCmd( _.getAddListenerBuilder
[spark] branch master updated: [SPARK-44559][PYTHON] Improve error messages for Python UDTF arrow cast
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 5384f4601a4 [SPARK-44559][PYTHON] Improve error messages for Python UDTF arrow cast 5384f4601a4 is described below commit 5384f4601a4ba8daba76d67e945eaa6fc2b70b2c Author: allisonwang-db AuthorDate: Mon Jul 31 11:55:47 2023 -0700 [SPARK-44559][PYTHON] Improve error messages for Python UDTF arrow cast ### What changes were proposed in this pull request? This PR improves error messages when the output of an arrow-optimized Python UDTF cannot be casted to the specified return schema of the UDTF. ### Why are the changes needed? To make Python UDTFs more user-friendly. ### Does this PR introduce _any_ user-facing change? Yes, before this PR, when the output of a UDTF fails to cast to the desired schema, Spark will throw this confusing error message: ```python udtf(returnType="x: int") class TestUDTF: def eval(self): yield [1, 2], TestUDTF().collect() ``` ``` File "pyarrow/array.pxi", line 1044, in pyarrow.lib.Array.from_pandas File "pyarrow/array.pxi", line 316, in pyarrow.lib.array File "pyarrow/array.pxi", line 83, in pyarrow.lib._ndarray_to_array File "pyarrow/error.pxi", line 100, in pyarrow.lib.check_status pyarrow.lib.ArrowInvalid: Could not convert [1, 2] with type list: tried to convert to int32 ``` Now, after this PR, the error message will look like this: `pyspark.errors.exceptions.base.PySparkRuntimeError: [UDTF_ARROW_TYPE_CAST_ERROR] Cannot convert the output value of the column 'x' with type 'object' to the specified return type of the column: 'int32'. Please check if the data types match and try again. ` ### How was this patch tested? New unit tests Closes #42191 from allisonwang-db/spark-44559-arrow-cast. Authored-by: allisonwang-db Signed-off-by: Takuya UESHIN --- python/pyspark/errors/error_classes.py | 5 + python/pyspark/sql/pandas/serializers.py | 69 +++- python/pyspark/sql/tests/test_udtf.py| 259 +++ 3 files changed, 332 insertions(+), 1 deletion(-) diff --git a/python/pyspark/errors/error_classes.py b/python/pyspark/errors/error_classes.py index c1b1d9eb7e1..f1396c49af0 100644 --- a/python/pyspark/errors/error_classes.py +++ b/python/pyspark/errors/error_classes.py @@ -720,6 +720,11 @@ ERROR_CLASSES_JSON = """ "Return type of the user-defined function should be , but is ." ] }, + "UDTF_ARROW_TYPE_CAST_ERROR" : { +"message" : [ + "Cannot convert the output value of the column '' with type '' to the specified return type of the column: ''. Please check if the data types match and try again." +] + }, "UDTF_EXEC_ERROR" : { "message" : [ "User defined table function encountered an error in the '' method: " diff --git a/python/pyspark/sql/pandas/serializers.py b/python/pyspark/sql/pandas/serializers.py index 15de00782c6..f3037c8b39c 100644 --- a/python/pyspark/sql/pandas/serializers.py +++ b/python/pyspark/sql/pandas/serializers.py @@ -19,7 +19,7 @@ Serializers for PyArrow and pandas conversions. See `pyspark.serializers` for more details. """ -from pyspark.errors import PySparkTypeError, PySparkValueError +from pyspark.errors import PySparkRuntimeError, PySparkTypeError, PySparkValueError from pyspark.serializers import Serializer, read_int, write_int, UTF8Deserializer, CPickleSerializer from pyspark.sql.pandas.types import ( from_arrow_type, @@ -538,6 +538,73 @@ class ArrowStreamPandasUDTFSerializer(ArrowStreamPandasUDFSerializer): return pa.RecordBatch.from_arrays(arrs, ["_%d" % i for i in range(len(arrs))]) +def _create_array(self, series, arrow_type, spark_type=None, arrow_cast=False): +""" +Override the `_create_array` method in the superclass to create an Arrow Array +from a given pandas.Series and an arrow type. The difference here is that we always +use arrow cast when creating the arrow array. Also, the error messages are specific +to arrow-optimized Python UDTFs. + +Parameters +-- +series : pandas.Series +A single series +arrow_type : pyarrow.DataType, optional +If None, pyarrow's inferred type will be used +spark_type : DataType, optional +If None, spark type converted from arrow_type will be used +arrow_cast: bool, optional +
[spark] branch branch-3.5 updated: [SPARK-44603] Add pyspark.testing to setup.py
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new 242ac2377ea [SPARK-44603] Add pyspark.testing to setup.py 242ac2377ea is described below commit 242ac2377ea14a62501ee7612e897cef686dadbe Author: Amanda Liu AuthorDate: Mon Jul 31 10:48:37 2023 -0700 [SPARK-44603] Add pyspark.testing to setup.py ### What changes were proposed in this pull request? This PR adds the pyspark.testing package to the Python setup.py file. ### Why are the changes needed? The change ensures that the PySpark test utils are available when users pip install PySpark. ### Does this PR introduce _any_ user-facing change? Yes, the PR allows users to use the PySpark test utils. ### How was this patch tested? Existing tests. Closes #42231 from asl3/testing-setup. Authored-by: Amanda Liu Signed-off-by: Takuya UESHIN (cherry picked from commit 1380a3df08e9fc9d7d50bc8dd5d7abee66d92510) Signed-off-by: Takuya UESHIN --- python/setup.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/setup.py b/python/setup.py index 3774273c421..a0297d4f9dc 100755 --- a/python/setup.py +++ b/python/setup.py @@ -263,6 +263,7 @@ try: "pyspark.pandas.usage_logging", "pyspark.python.pyspark", "pyspark.python.lib", +"pyspark.testing", "pyspark.data", "pyspark.licenses", "pyspark.resource", - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-44603] Add pyspark.testing to setup.py
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 1380a3df08e [SPARK-44603] Add pyspark.testing to setup.py 1380a3df08e is described below commit 1380a3df08e9fc9d7d50bc8dd5d7abee66d92510 Author: Amanda Liu AuthorDate: Mon Jul 31 10:48:37 2023 -0700 [SPARK-44603] Add pyspark.testing to setup.py ### What changes were proposed in this pull request? This PR adds the pyspark.testing package to the Python setup.py file. ### Why are the changes needed? The change ensures that the PySpark test utils are available when users pip install PySpark. ### Does this PR introduce _any_ user-facing change? Yes, the PR allows users to use the PySpark test utils. ### How was this patch tested? Existing tests. Closes #42231 from asl3/testing-setup. Authored-by: Amanda Liu Signed-off-by: Takuya UESHIN --- python/setup.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/setup.py b/python/setup.py index fa938b0b4ef..dc46271eea8 100755 --- a/python/setup.py +++ b/python/setup.py @@ -264,6 +264,7 @@ try: "pyspark.pandas.usage_logging", "pyspark.python.pyspark", "pyspark.python.lib", +"pyspark.testing", "pyspark.data", "pyspark.licenses", "pyspark.resource", - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-43968][PYTHON] Improve error messages for Python UDTFs with wrong number of outputs
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 7194ce9263f [SPARK-43968][PYTHON] Improve error messages for Python UDTFs with wrong number of outputs 7194ce9263f is described below commit 7194ce9263fe1683c039a1aaf9462657b1672a99 Author: allisonwang-db AuthorDate: Thu Jul 27 13:18:39 2023 -0700 [SPARK-43968][PYTHON] Improve error messages for Python UDTFs with wrong number of outputs ### What changes were proposed in this pull request? This PR improves the error messages for Python UDTFs when the number of outputs mismatches the number of outputs specified in the return type of the UDTFs. ### Why are the changes needed? To make Python UDTFs more user-friendly. ### Does this PR introduce _any_ user-facing change? Yes. This PR improves the error messages. Before this change, the error thrown by Spark will be a java IllegalStateException: ``` java.lang.IllegalStateException: Input row doesn't have expected number of values required by the schema ``` After this PR, it will throw a clearer error message with an error class: ``` [UDTF_RETURN_SCHEMA_MISMATCH] The number of columns in the result does not match the specified schema ``` ### How was this patch tested? Existing tests and new unit tests. Closes #42157 from allisonwang-db/spark-43968-py-udtf-checks. Authored-by: allisonwang-db Signed-off-by: Takuya UESHIN --- python/pyspark/errors/error_classes.py | 5 + python/pyspark/sql/connect/udtf.py | 4 +- .../pyspark/sql/tests/connect/test_parity_udtf.py | 50 python/pyspark/sql/tests/test_udtf.py | 133 +++-- python/pyspark/sql/udtf.py | 9 +- python/pyspark/worker.py | 22 +++- 6 files changed, 99 insertions(+), 124 deletions(-) diff --git a/python/pyspark/errors/error_classes.py b/python/pyspark/errors/error_classes.py index e0d1c30b604..f4b643f1d32 100644 --- a/python/pyspark/errors/error_classes.py +++ b/python/pyspark/errors/error_classes.py @@ -283,6 +283,11 @@ ERROR_CLASSES_JSON = """ "The eval type for the UDTF '' is invalid. It must be one of ." ] }, + "INVALID_UDTF_HANDLER_TYPE" : { +"message" : [ + "The UDTF is invalid. The function handler must be a class, but got ''. Please provide a class as the function handler." +] + }, "INVALID_UDTF_NO_EVAL" : { "message" : [ "The UDTF '' is invalid. It does not implement the required 'eval' method. Please implement the 'eval' method in '' and try again." diff --git a/python/pyspark/sql/connect/udtf.py b/python/pyspark/sql/connect/udtf.py index 74c55cc42cd..919994401c8 100644 --- a/python/pyspark/sql/connect/udtf.py +++ b/python/pyspark/sql/connect/udtf.py @@ -124,6 +124,8 @@ class UserDefinedTableFunction: evalType: int = PythonEvalType.SQL_TABLE_UDF, deterministic: bool = True, ) -> None: +_validate_udtf_handler(func, returnType) + self.func = func self.returnType: Optional[DataType] = ( None @@ -136,8 +138,6 @@ class UserDefinedTableFunction: self.evalType = evalType self.deterministic = deterministic -_validate_udtf_handler(func, returnType) - def _build_common_inline_user_defined_table_function( self, *cols: "ColumnOrName" ) -> CommonInlineUserDefinedTableFunction: diff --git a/python/pyspark/sql/tests/connect/test_parity_udtf.py b/python/pyspark/sql/tests/connect/test_parity_udtf.py index 1aff1bd0686..748b611e667 100644 --- a/python/pyspark/sql/tests/connect/test_parity_udtf.py +++ b/python/pyspark/sql/tests/connect/test_parity_udtf.py @@ -56,56 +56,6 @@ class UDTFParityTests(BaseUDTFTestsMixin, ReusedConnectTestCase): ): TestUDTF(lit(1)).collect() -def test_udtf_with_wrong_num_output(self): -err_msg = ( -"java.lang.IllegalStateException: Input row doesn't have expected number of " -+ "values required by the schema." -) - -@udtf(returnType="a: int, b: int") -class TestUDTF: -def eval(self, a: int): -yield a, - -with self.assertRaisesRegex(SparkConnectGrpcException, err_msg): -TestUDTF(lit(1)).collect() - -@udtf(returnType="a: int") -class TestUDTF: -def eval(self, a: int): -yield a, a + 1 - -with self.assertRaisesRegex(SparkConnectGrpcException
[spark] branch master updated: [SPARK-44533][PYTHON] Add support for accumulator, broadcast, and Spark files in Python UDTF's analyze
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 8647b243dee [SPARK-44533][PYTHON] Add support for accumulator, broadcast, and Spark files in Python UDTF's analyze 8647b243dee is described below commit 8647b243deed8f2c3279ed17fe196006b6c923af Author: Takuya UESHIN AuthorDate: Wed Jul 26 21:03:08 2023 -0700 [SPARK-44533][PYTHON] Add support for accumulator, broadcast, and Spark files in Python UDTF's analyze ### What changes were proposed in this pull request? Adds support for `accumulator`, `broadcast` in vanilla PySpark, and Spark files in both vanilla PySpark and Spark Connect Python client, in Python UDTF's analyze. For example, in vanilla PySpark: ```py >>> colname = sc.broadcast("col1") >>> test_accum = sc.accumulator(0) >>> udtf ... class TestUDTF: ... staticmethod ... def analyze(a: AnalyzeArgument) -> AnalyzeResult: ... test_accum.add(1) ... return AnalyzeResult(StructType().add(colname.value, a.data_type)) ... def eval(self, a): ... test_accum.add(1) ... yield a, ... >>> df = TestUDTF(lit(10)) >>> df.printSchema() root |-- col1: integer (nullable = true) >>> df.show() ++ |col1| ++ | 10| ++ >>> test_accum.value 2 ``` or ```py >>> pyfile_path = "my_pyfile.py" >>> with open(pyfile_path, "w") as f: ... f.write("my_func = lambda: 'col1'") ... 24 >>> sc.addPyFile(pyfile_path) >>> # or spark.addArtifacts(pyfile_path, pyfile=True) >>> >>> udtf ... class TestUDTF: ... staticmethod ... def analyze(a: AnalyzeArgument) -> AnalyzeResult: ... import my_pyfile ... return AnalyzeResult(StructType().add(my_pyfile.my_func(), a.data_type)) ... def eval(self, a): ... yield a, ... >>> df = TestUDTF(lit(10)) >>> df.printSchema() root |-- col1: integer (nullable = true) >>> df.show() ++ |col1| ++ | 10| ++ ``` ### Why are the changes needed? To support missing features: `accumulator`, `broadcast`, and Spark files in Python UDTF's analyze. ### Does this PR introduce _any_ user-facing change? Yes, accumulator, broadcast in vanilla PySpark, and Spark files in both vanilla PySpark and Spark Connect Python client will be available. ### How was this patch tested? Added related tests. Closes #42135 from ueshin/issues/SPARK-44533/analyze. Authored-by: Takuya UESHIN Signed-off-by: Takuya UESHIN --- .../org/apache/spark/api/python/PythonRDD.scala| 4 +- .../org/apache/spark/api/python/PythonRunner.scala | 83 +--- .../spark/api/python/PythonWorkerUtils.scala | 152 ++ .../pyspark/sql/tests/connect/test_parity_udtf.py | 19 ++ python/pyspark/sql/tests/test_udtf.py | 224 - python/pyspark/sql/worker/analyze_udtf.py | 18 +- python/pyspark/worker.py | 91 ++--- python/pyspark/worker_util.py | 132 .../execution/python/BatchEvalPythonUDTFExec.scala | 7 +- .../python/UserDefinedPythonFunction.scala | 23 ++- 10 files changed, 584 insertions(+), 169 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 95fbc145d83..91fd92d4422 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -487,9 +487,7 @@ private[spark] object PythonRDD extends Logging { } def writeUTF(str: String, dataOut: DataOutputStream): Unit = { -val bytes = str.getBytes(StandardCharsets.UTF_8) -dataOut.writeInt(bytes.length) -dataOut.write(bytes) +PythonWorkerUtils.writeUTF(str, dataOut) } /** diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala index 5d719b33a30..0173de75ff2 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala @@ -309,8 +309,9 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( val dataOut = new DataOutputStream(stream) // Partition index dataOut.w
[spark] branch branch-3.5 updated: [SPARK-44479][PYTHON][3.5] Fix ArrowStreamPandasUDFSerializer to accept no-column pandas DataFrame
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new 803b2854a9e [SPARK-44479][PYTHON][3.5] Fix ArrowStreamPandasUDFSerializer to accept no-column pandas DataFrame 803b2854a9e is described below commit 803b2854a9e82aee4e5691c4a9a697856b963377 Author: Takuya UESHIN AuthorDate: Wed Jul 26 17:54:38 2023 -0700 [SPARK-44479][PYTHON][3.5] Fix ArrowStreamPandasUDFSerializer to accept no-column pandas DataFrame ### What changes were proposed in this pull request? Fixes `ArrowStreamPandasUDFSerializer` to accept no-column pandas DataFrame. ```py >>> def _scalar_f(id): ... return pd.DataFrame(index=id) ... >>> scalar_f = pandas_udf(_scalar_f, returnType=StructType()) >>> df = spark.range(3).withColumn("f", scalar_f(col("id"))) >>> df.printSchema() root |-- id: long (nullable = false) |-- f: struct (nullable = true) >>> df.show() +---+---+ | id| f| +---+---+ | 0| {}| | 1| {}| | 2| {}| +---+---+ ``` ### Why are the changes needed? The above query fails with the following error: ```py >>> df.show() org.apache.spark.api.python.PythonException: Traceback (most recent call last): ... ValueError: not enough values to unpack (expected 2, got 0) ``` ### Does this PR introduce _any_ user-facing change? Yes, Pandas UDF will accept no-column pandas DataFrame. ### How was this patch tested? Added related tests. Closes #42176 from ueshin/issues/SPARK-44479/3.5/empty_schema. Authored-by: Takuya UESHIN Signed-off-by: Takuya UESHIN --- python/pyspark/sql/connect/types.py| 4 ++- python/pyspark/sql/pandas/serializers.py | 31 -- .../sql/tests/pandas/test_pandas_udf_scalar.py | 23 +++- python/pyspark/sql/tests/test_udtf.py | 11 ++-- 4 files changed, 45 insertions(+), 24 deletions(-) diff --git a/python/pyspark/sql/connect/types.py b/python/pyspark/sql/connect/types.py index 2a21cdf0675..0db2833d2c1 100644 --- a/python/pyspark/sql/connect/types.py +++ b/python/pyspark/sql/connect/types.py @@ -170,6 +170,7 @@ def pyspark_types_to_proto_types(data_type: DataType) -> pb2.DataType: ret.year_month_interval.start_field = data_type.startField ret.year_month_interval.end_field = data_type.endField elif isinstance(data_type, StructType): +struct = pb2.DataType.Struct() for field in data_type.fields: struct_field = pb2.DataType.StructField() struct_field.name = field.name @@ -177,7 +178,8 @@ def pyspark_types_to_proto_types(data_type: DataType) -> pb2.DataType: struct_field.nullable = field.nullable if field.metadata is not None and len(field.metadata) > 0: struct_field.metadata = json.dumps(field.metadata) -ret.struct.fields.append(struct_field) +struct.fields.append(struct_field) +ret.struct.CopyFrom(struct) elif isinstance(data_type, MapType): ret.map.key_type.CopyFrom(pyspark_types_to_proto_types(data_type.keyType)) ret.map.value_type.CopyFrom(pyspark_types_to_proto_types(data_type.valueType)) diff --git a/python/pyspark/sql/pandas/serializers.py b/python/pyspark/sql/pandas/serializers.py index f22a73cbbef..1d326928e23 100644 --- a/python/pyspark/sql/pandas/serializers.py +++ b/python/pyspark/sql/pandas/serializers.py @@ -385,37 +385,28 @@ class ArrowStreamPandasUDFSerializer(ArrowStreamPandasSerializer): """ import pyarrow as pa -# Input partition and result pandas.DataFrame empty, make empty Arrays with struct -if len(df) == 0 and len(df.columns) == 0: -arrs_names = [ -(pa.array([], type=field.type), field.name) for field in arrow_struct_type -] +if len(df.columns) == 0: +return pa.array([{}] * len(df), arrow_struct_type) # Assign result columns by schema name if user labeled with strings -elif self._assign_cols_by_name and any(isinstance(name, str) for name in df.columns): -arrs_names = [ -( -self._create_array(df[field.name], field.type, arrow_cast=self._arrow_cast), -field.name, -) +if self._assign_cols_by_name and any(isinstance(name, str) for name in df.columns): +struct_arrs = [ +self._create_array(df[field.name], field.type, arrow_cast=self._arrow_cast) for field in arrow_struct_type
[spark] branch master updated: [SPARK-44479][PYTHON] Fix ArrowStreamPandasUDFSerializer to accept no-column pandas DataFrame
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 02e36dd0f07 [SPARK-44479][PYTHON] Fix ArrowStreamPandasUDFSerializer to accept no-column pandas DataFrame 02e36dd0f07 is described below commit 02e36dd0f077d11a75c6e083489dc1a51c870a0d Author: Takuya UESHIN AuthorDate: Wed Jul 26 17:53:46 2023 -0700 [SPARK-44479][PYTHON] Fix ArrowStreamPandasUDFSerializer to accept no-column pandas DataFrame ### What changes were proposed in this pull request? Fixes `ArrowStreamPandasUDFSerializer` to accept no-column pandas DataFrame. ```py >>> def _scalar_f(id): ... return pd.DataFrame(index=id) ... >>> scalar_f = pandas_udf(_scalar_f, returnType=StructType()) >>> df = spark.range(3).withColumn("f", scalar_f(col("id"))) >>> df.printSchema() root |-- id: long (nullable = false) |-- f: struct (nullable = true) >>> df.show() +---+---+ | id| f| +---+---+ | 0| {}| | 1| {}| | 2| {}| +---+---+ ``` ### Why are the changes needed? The above query fails with the following error: ```py >>> df.show() org.apache.spark.api.python.PythonException: Traceback (most recent call last): ... ValueError: not enough values to unpack (expected 2, got 0) ``` ### Does this PR introduce _any_ user-facing change? Yes, Pandas UDF will accept no-column pandas DataFrame. ### How was this patch tested? Added related tests. Closes #42161 from ueshin/issues/SPARK-44479/empty_schema. Authored-by: Takuya UESHIN Signed-off-by: Takuya UESHIN --- python/pyspark/sql/connect/types.py| 4 ++- python/pyspark/sql/pandas/serializers.py | 31 -- .../sql/tests/pandas/test_pandas_udf_scalar.py | 23 +++- python/pyspark/sql/tests/test_udtf.py | 11 ++-- 4 files changed, 45 insertions(+), 24 deletions(-) diff --git a/python/pyspark/sql/connect/types.py b/python/pyspark/sql/connect/types.py index 2a21cdf0675..0db2833d2c1 100644 --- a/python/pyspark/sql/connect/types.py +++ b/python/pyspark/sql/connect/types.py @@ -170,6 +170,7 @@ def pyspark_types_to_proto_types(data_type: DataType) -> pb2.DataType: ret.year_month_interval.start_field = data_type.startField ret.year_month_interval.end_field = data_type.endField elif isinstance(data_type, StructType): +struct = pb2.DataType.Struct() for field in data_type.fields: struct_field = pb2.DataType.StructField() struct_field.name = field.name @@ -177,7 +178,8 @@ def pyspark_types_to_proto_types(data_type: DataType) -> pb2.DataType: struct_field.nullable = field.nullable if field.metadata is not None and len(field.metadata) > 0: struct_field.metadata = json.dumps(field.metadata) -ret.struct.fields.append(struct_field) +struct.fields.append(struct_field) +ret.struct.CopyFrom(struct) elif isinstance(data_type, MapType): ret.map.key_type.CopyFrom(pyspark_types_to_proto_types(data_type.keyType)) ret.map.value_type.CopyFrom(pyspark_types_to_proto_types(data_type.valueType)) diff --git a/python/pyspark/sql/pandas/serializers.py b/python/pyspark/sql/pandas/serializers.py index 90a24197f64..15de00782c6 100644 --- a/python/pyspark/sql/pandas/serializers.py +++ b/python/pyspark/sql/pandas/serializers.py @@ -385,37 +385,28 @@ class ArrowStreamPandasUDFSerializer(ArrowStreamPandasSerializer): """ import pyarrow as pa -# Input partition and result pandas.DataFrame empty, make empty Arrays with struct -if len(df) == 0 and len(df.columns) == 0: -arrs_names = [ -(pa.array([], type=field.type), field.name) for field in arrow_struct_type -] +if len(df.columns) == 0: +return pa.array([{}] * len(df), arrow_struct_type) # Assign result columns by schema name if user labeled with strings -elif self._assign_cols_by_name and any(isinstance(name, str) for name in df.columns): -arrs_names = [ -( -self._create_array(df[field.name], field.type, arrow_cast=self._arrow_cast), -field.name, -) +if self._assign_cols_by_name and any(isinstance(name, str) for name in df.columns): +struct_arrs = [ +self._create_array(df[field.name], field.type, arrow_cast=self._arrow_cast) for field in arrow_struct_type ] # Ass
[spark] branch master updated: [SPARK-44503][SQL][FOLLOWUP] Simplify the test case for PARTITION BY and ORDER BY clause after TABLE arguments for TVF calls
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new a8b25f8b5ba [SPARK-44503][SQL][FOLLOWUP] Simplify the test case for PARTITION BY and ORDER BY clause after TABLE arguments for TVF calls a8b25f8b5ba is described below commit a8b25f8b5ba2304dff5e78b1001923f58997bb1f Author: Jiaan Geng AuthorDate: Tue Jul 25 10:39:43 2023 -0700 [SPARK-44503][SQL][FOLLOWUP] Simplify the test case for PARTITION BY and ORDER BY clause after TABLE arguments for TVF calls ### What changes were proposed in this pull request? https://github.com/apache/spark/pull/42100 added SQL grammar for PARTITION BY and ORDER BY clause after TABLE arguments for TVF calls. Because I reviewed it later, so I commit this PR. ### Why are the changes needed? Simplify the test case for PARTITION BY and ORDER BY clause after TABLE arguments for TVF calls ### Does this PR introduce _any_ user-facing change? 'No'. New feature. ### How was this patch tested? N/A. Closes #42148 from beliefer/SPARK-44503. Authored-by: Jiaan Geng Signed-off-by: Takuya UESHIN --- .../sql/catalyst/parser/PlanParserSuite.scala | 66 -- 1 file changed, 23 insertions(+), 43 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index 1d50e3bb479..1f50cadacb0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -1493,52 +1493,32 @@ class PlanParserSuite extends AnalysisTest { } test("SPARK-44503: Support PARTITION BY and ORDER BY clause for TVF TABLE arguments") { +val message = "Specifying the PARTITION BY clause for TABLE arguments is not implemented yet" +val startIndex = 29 + +def check(sqlSuffix: String): Unit = { + val sql = s"select * from my_tvf(arg1 => $sqlSuffix)" + checkError( +exception = parseException(sql), +errorClass = "_LEGACY_ERROR_TEMP_0035", +parameters = Map("message" -> message), +context = ExpectedContext( + fragment = sqlSuffix, + start = startIndex, + stop = sql.length - 2)) +} + Seq("partition", "distribute").foreach { partition => + val sqlSuffix = s"table(v1) $partition by col1" + check(sqlSuffix) + Seq("order", "sort").foreach { order => -val sql1suffix = s"table(v1) $partition by col1" -val sql1 = s"select * from my_tvf(arg1 => $sql1suffix)" -val startIndex = 29 -val message = - "Specifying the PARTITION BY clause for TABLE arguments is not implemented yet" -checkError( - exception = parseException(sql1), - errorClass = "_LEGACY_ERROR_TEMP_0035", - parameters = Map("message" -> message), - context = ExpectedContext( -fragment = sql1suffix, -start = startIndex, -stop = sql1.length - 2)) -val sql2suffix = s"table(v1) $partition by col1 $order by col2 asc" -val sql2 = s"select * from my_tvf(arg1 => $sql2suffix)" -checkError( - exception = parseException(sql2), - errorClass = "_LEGACY_ERROR_TEMP_0035", - parameters = Map("message" -> message), - context = ExpectedContext( -fragment = sql2suffix, -start = startIndex, -stop = sql2.length - 2)) -val sql3suffix = s"table(v1) $partition by col1, col2 $order by col2 asc, col3 desc" -val sql3 = s"select * from my_tvf(arg1 => $sql3suffix)" -checkError( - exception = parseException(sql3), - errorClass = "_LEGACY_ERROR_TEMP_0035", - parameters = Map("message" -> message), - context = ExpectedContext( -fragment = sql3suffix, -start = startIndex, -stop = sql3.length - 2)) -val sql4Suffix = s"table(select col1, col2, col3 from v2) $partition by col1, col2 " + +Seq( + s"table(v1) $partition by col1 $order by col2 asc", + s"table(v1) $partition by col1, col2 $order by col2 asc, col3 desc", + s"table(select col1, col2, col3 from v2) $partition by col1, col2 " + s"$order by col2 asc, col3 desc" -val sql4
[spark] branch master updated: [SPARK-44503][SQL] Add SQL grammar for PARTITION BY and ORDER BY clause after TABLE arguments for TVF calls
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 58e5d86cc07 [SPARK-44503][SQL] Add SQL grammar for PARTITION BY and ORDER BY clause after TABLE arguments for TVF calls 58e5d86cc07 is described below commit 58e5d86cc076d4546dac5e1f594977d615ec1e7a Author: Daniel Tenedorio AuthorDate: Mon Jul 24 16:08:46 2023 -0700 [SPARK-44503][SQL] Add SQL grammar for PARTITION BY and ORDER BY clause after TABLE arguments for TVF calls ### What changes were proposed in this pull request? This PR adds SQL grammar for PARTITION BY and ORDER BY clause after TABLE arguments for TVF calls. Note that this PR just includes the SQL grammar and parsing part, no analysis support is implemented yet, that will come next. Examples: ``` select * from tvf(arg1 => table(t1) partition by col1); select * from tvf(arg1 => table(t1) partition by col1 order by col2 asc); select * from tvf(arg1 => table(t1) partition by col1, col2 order by col2 asc, col3 desc); select * from tvf(arg1 => table(select col1, col2, col3 from v2) partition by col1, col2 order by col2 asc, col3 desc); ``` ### Why are the changes needed? This will provide a way for the TVF caller to indicate desired semantics for dividing up the rows of the input table into partitions for consumption by the underlying algorithm. ### Does this PR introduce _any_ user-facing change? Yes, it adds new SQL grammar. ### How was this patch tested? This PR adds new parser unit tests. Currently the parser returns "not implemented yet" error for these cases, and we will implement analysis for them next. Closes #42100 from dtenedor/partition-by-clause. Authored-by: Daniel Tenedorio Signed-off-by: Takuya UESHIN --- .../spark/sql/catalyst/parser/SqlBaseParser.g4 | 11 +++-- .../spark/sql/catalyst/parser/AstBuilder.scala | 7 +++ .../sql/catalyst/parser/PlanParserSuite.scala | 51 ++ 3 files changed, 66 insertions(+), 3 deletions(-) diff --git a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 index 85dbc499fbd..372e0b54732 100644 --- a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 +++ b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 @@ -789,9 +789,14 @@ inlineTable ; functionTableSubqueryArgument -: TABLE identifierReference -| TABLE LEFT_PAREN identifierReference RIGHT_PAREN -| TABLE LEFT_PAREN query RIGHT_PAREN +: TABLE identifierReference tableArgumentPartitioning? +| TABLE LEFT_PAREN identifierReference RIGHT_PAREN tableArgumentPartitioning? +| TABLE LEFT_PAREN query RIGHT_PAREN tableArgumentPartitioning? +; + +tableArgumentPartitioning +: (PARTITION | DISTRIBUTE) BY expressionSeq + ((ORDER | SORT) BY sortItem (COMMA sortItem)*)? ; functionTableNamedArgumentExpression diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 7a28efa3e42..ccfcd13440c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1564,6 +1564,13 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging { }.getOrElse { plan(ctx.query) } +val partitioning = Option(ctx.tableArgumentPartitioning) +if (partitioning.isDefined) { + // The PARTITION BY clause is not implemented yet for TABLE arguments to table valued function + // calls. + operationNotAllowed( +"Specifying the PARTITION BY clause for TABLE arguments is not implemented yet", ctx) +} FunctionTableSubqueryArgumentExpression(p) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index 4a5d0a0ae29..1d50e3bb479 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -1492,6 +1492,57 @@ class PlanParserSuite extends AnalysisTest { stop = sql1.length - 2)) } + test("SPARK-44503: Support PARTITION BY and ORDER BY clause for TVF TABLE arguments") { +Seq("partition", "distribute").foreach { partition => + Seq("order", "so
[spark] branch branch-3.5 updated: Revert "[SPARK-44380][PYTHON][FOLLOWUP] Set __doc__ for analyze static method when Arrow is enabled"
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new d0ad48334dc Revert "[SPARK-44380][PYTHON][FOLLOWUP] Set __doc__ for analyze static method when Arrow is enabled" d0ad48334dc is described below commit d0ad48334dccb451a0312c45ffe142b533ae75f9 Author: Takuya UESHIN AuthorDate: Mon Jul 24 10:11:18 2023 -0700 Revert "[SPARK-44380][PYTHON][FOLLOWUP] Set __doc__ for analyze static method when Arrow is enabled" This reverts commit 07d3e8e52878ea9631d4757b67119a18fbdf0230. --- python/pyspark/sql/tests/test_udtf.py | 8 +--- python/pyspark/sql/udtf.py| 3 --- 2 files changed, 1 insertion(+), 10 deletions(-) diff --git a/python/pyspark/sql/tests/test_udtf.py b/python/pyspark/sql/tests/test_udtf.py index 0fdb1c9b8a1..2c76d2f7e15 100644 --- a/python/pyspark/sql/tests/test_udtf.py +++ b/python/pyspark/sql/tests/test_udtf.py @@ -728,11 +728,6 @@ class BaseUDTFTestsMixin: """Initialize the UDTF""" ... -@staticmethod -def analyze(x: AnalyzeArgument) -> AnalyzeResult: -"""Analyze the argument.""" -... - def eval(self, x: int): """Evaluate the input row.""" yield x + 1, @@ -741,10 +736,9 @@ class BaseUDTFTestsMixin: """Terminate the UDTF.""" ... -cls = udtf(TestUDTF).func +cls = udtf(TestUDTF, returnType="y: int").func self.assertIn("A UDTF for test", cls.__doc__) self.assertIn("Initialize the UDTF", cls.__init__.__doc__) -self.assertIn("Analyze the argument", cls.analyze.__doc__) self.assertIn("Evaluate the input row", cls.eval.__doc__) self.assertIn("Terminate the UDTF", cls.terminate.__doc__) diff --git a/python/pyspark/sql/udtf.py b/python/pyspark/sql/udtf.py index e930daa9f51..3ab74193093 100644 --- a/python/pyspark/sql/udtf.py +++ b/python/pyspark/sql/udtf.py @@ -134,9 +134,6 @@ def _vectorize_udtf(cls: Type) -> Type: if hasattr(cls, "terminate"): getattr(vectorized_udtf, "terminate").__doc__ = getattr(cls, "terminate").__doc__ -if hasattr(vectorized_udtf, "analyze"): -getattr(vectorized_udtf, "analyze").__doc__ = getattr(cls, "analyze").__doc__ - return vectorized_udtf - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-44395][SQL] Update TVF arguments to require parentheses around identifier after TABLE keyword
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 63cef16ef77 [SPARK-44395][SQL] Update TVF arguments to require parentheses around identifier after TABLE keyword 63cef16ef77 is described below commit 63cef16ef779addaa841fa863a5797fc9f33fd82 Author: Daniel Tenedorio AuthorDate: Thu Jul 13 14:04:09 2023 -0700 [SPARK-44395][SQL] Update TVF arguments to require parentheses around identifier after TABLE keyword ### What changes were proposed in this pull request? This PR updates the parsing of table function arguments to require parentheses around identifier after the TABLE keyword. Instead of `TABLE t`, the syntax should look like `TABLE(t)` instead as specified in the SQL standard. * I kept the previous syntax without the parentheses as an optional case in the SQL grammar so that we can catch it in the `AstBuilder` and throw an informative error message telling the user to add parentheses and try the query again. * I had to swap the order of parsing table function arguments, so the `table(identifier)` syntax does not accidentally parse as a scalar function call: ``` functionTableArgument : functionTableReferenceArgument | functionArgument ; ``` ### Why are the changes needed? This syntax is written down in the SQL standard. Per the standard, `TABLE identifier` should actually be passed as `TABLE(identifier)`. ### Does this PR introduce _any_ user-facing change? Yes, SQL syntax changes slightly. ### How was this patch tested? This PR adds and updates unit test coverage. Closes #41965 from dtenedor/parentheses-table-clause. Authored-by: Daniel Tenedorio Signed-off-by: Takuya UESHIN --- .../src/main/resources/error/error-classes.json | 5 + ...ror-conditions-invalid-sql-syntax-error-class.md | 4 python/pyspark/sql/tests/test_udtf.py | 10 +- .../spark/sql/catalyst/parser/SqlBaseParser.g4 | 5 +++-- .../spark/sql/catalyst/parser/AstBuilder.scala | 6 ++ .../spark/sql/errors/QueryParsingErrors.scala | 11 +++ .../spark/sql/catalyst/parser/PlanParserSuite.scala | 21 + 7 files changed, 51 insertions(+), 11 deletions(-) diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index 6691e86b463..99a0a4ae4ba 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -1694,6 +1694,11 @@ "Expected a column reference for transform : ." ] }, + "INVALID_TABLE_FUNCTION_IDENTIFIER_ARGUMENT_MISSING_PARENTHESES" : { +"message" : [ + "Syntax error: call to table-valued function is invalid because parentheses are missing around the provided TABLE argument ; please surround this with parentheses and try again." +] + }, "INVALID_TABLE_VALUED_FUNC_NAME" : { "message" : [ "Table valued function cannot specify database name: ." diff --git a/docs/sql-error-conditions-invalid-sql-syntax-error-class.md b/docs/sql-error-conditions-invalid-sql-syntax-error-class.md index 6c9f588ba49..b1e298f7b90 100644 --- a/docs/sql-error-conditions-invalid-sql-syntax-error-class.md +++ b/docs/sql-error-conditions-invalid-sql-syntax-error-class.md @@ -49,6 +49,10 @@ Partition key `` must set value. Expected a column reference for transform ``: ``. +## INVALID_TABLE_FUNCTION_IDENTIFIER_ARGUMENT_MISSING_PARENTHESES + +Syntax error: call to table-valued function is invalid because parentheses are missing around the provided TABLE argument ``; please surround this with parentheses and try again. + ## INVALID_TABLE_VALUED_FUNC_NAME Table valued function cannot specify database name: ``. diff --git a/python/pyspark/sql/tests/test_udtf.py b/python/pyspark/sql/tests/test_udtf.py index e4db542cbb7..8f42b4123e4 100644 --- a/python/pyspark/sql/tests/test_udtf.py +++ b/python/pyspark/sql/tests/test_udtf.py @@ -545,7 +545,7 @@ class BaseUDTFTestsMixin: with self.tempView("v"): self.spark.sql("CREATE OR REPLACE TEMPORARY VIEW v as SELECT id FROM range(0, 8)") self.assertEqual( -self.spark.sql("SELECT * FROM test_udtf(TABLE v)").collect(), +self.spark.sql("SELECT * FROM test_udtf(TABLE (v))").collect(), [Row(a=6), Row(a=7)], ) @@ -561,7 +561,7 @@ class BaseUDTFTestsMixin: with self.tempView("v"): self.spark.sql("CREAT
[spark] branch master updated (a8893422752 -> 47ddac13144)
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from a8893422752 [SPARK-43867][SQL] Improve suggested candidates for unresolved attribute add 47ddac13144 [SPARK-43804][PYTHON][TESTS] Test on nested structs support in Pandas UDF No new revisions were added by this update. Summary of changes: .../sql/tests/connect/test_parity_pandas_udf.py| 3 - .../sql/tests/pandas/test_pandas_udf_scalar.py | 107 + 2 files changed, 89 insertions(+), 21 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-42437][CONNECT][PYTHON][FOLLOW-UP] Storage level proto converters
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new c291564c7d4 [SPARK-42437][CONNECT][PYTHON][FOLLOW-UP] Storage level proto converters c291564c7d4 is described below commit c291564c7d493a9da5e2315d6bab28796dfce7ce Author: khalidmammadov AuthorDate: Wed Apr 19 13:30:53 2023 -0700 [SPARK-42437][CONNECT][PYTHON][FOLLOW-UP] Storage level proto converters ### What changes were proposed in this pull request? Converters between Proto and StorageLevel to avoid code duplication It's follow up from https://github.com/apache/spark/pull/40015 ### Why are the changes needed? Code deduplication ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing tests Closes #40859 from khalidmammadov/storage_level_converter. Authored-by: khalidmammadov Signed-off-by: Takuya UESHIN --- python/pyspark/sql/connect/client.py | 19 +++ python/pyspark/sql/connect/conversion.py | 24 python/pyspark/sql/connect/plan.py | 11 ++- 3 files changed, 29 insertions(+), 25 deletions(-) diff --git a/python/pyspark/sql/connect/client.py b/python/pyspark/sql/connect/client.py index 780c5702477..60f3f1ac2ba 100644 --- a/python/pyspark/sql/connect/client.py +++ b/python/pyspark/sql/connect/client.py @@ -60,6 +60,7 @@ from google.protobuf import text_format from google.rpc import error_details_pb2 from pyspark.resource.information import ResourceInformation +from pyspark.sql.connect.conversion import storage_level_to_proto, proto_to_storage_level import pyspark.sql.connect.proto as pb2 import pyspark.sql.connect.proto.base_pb2_grpc as grpc_lib import pyspark.sql.connect.types as types @@ -469,13 +470,7 @@ class AnalyzeResult: elif pb.HasField("unpersist"): pass elif pb.HasField("get_storage_level"): -storage_level = StorageLevel( -useDisk=pb.get_storage_level.storage_level.use_disk, -useMemory=pb.get_storage_level.storage_level.use_memory, -useOffHeap=pb.get_storage_level.storage_level.use_off_heap, -deserialized=pb.get_storage_level.storage_level.deserialized, -replication=pb.get_storage_level.storage_level.replication, -) +storage_level = proto_to_storage_level(pb.get_storage_level.storage_level) else: raise SparkConnectException("No analyze result found!") @@ -877,15 +872,7 @@ class SparkConnectClient(object): req.persist.relation.CopyFrom(cast(pb2.Relation, kwargs.get("relation"))) if kwargs.get("storage_level", None) is not None: storage_level = cast(StorageLevel, kwargs.get("storage_level")) -req.persist.storage_level.CopyFrom( -pb2.StorageLevel( -use_disk=storage_level.useDisk, -use_memory=storage_level.useMemory, -use_off_heap=storage_level.useOffHeap, -deserialized=storage_level.deserialized, -replication=storage_level.replication, -) -) + req.persist.storage_level.CopyFrom(storage_level_to_proto(storage_level)) elif method == "unpersist": req.unpersist.relation.CopyFrom(cast(pb2.Relation, kwargs.get("relation"))) if kwargs.get("blocking", None) is not None: diff --git a/python/pyspark/sql/connect/conversion.py b/python/pyspark/sql/connect/conversion.py index 5a31d1df67e..a6fe0c00e09 100644 --- a/python/pyspark/sql/connect/conversion.py +++ b/python/pyspark/sql/connect/conversion.py @@ -43,7 +43,9 @@ from pyspark.sql.types import ( cast, ) +from pyspark.storagelevel import StorageLevel from pyspark.sql.connect.types import to_arrow_schema +import pyspark.sql.connect.proto as pb2 from typing import ( Any, @@ -486,3 +488,25 @@ class ArrowTableToRowsConversion: values = [field_converters[j](columnar_data[j][i]) for j in range(table.num_columns)] rows.append(_create_row(fields=schema.fieldNames(), values=values)) return rows + + +def storage_level_to_proto(storage_level: StorageLevel) -> pb2.StorageLevel: +assert storage_level is not None and isinstance(storage_level, StorageLevel) +return pb2.StorageLevel( +use_disk=storage_level.useDisk, +use_memory=storage_level.useMemory, +use_off_heap=storage_level.useOffHeap, +deserialized=storage_level.deserialized, +replication=storage_level.replication, +
[spark] branch master updated (dabd771c37b -> 2931993e059)
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from dabd771c37b [SPARK-43038][SQL] Support the CBC mode by `aes_encrypt()`/`aes_decrypt()` add 2931993e059 [SPARK-42437][PYTHON][CONNECT] PySpark catalog.cacheTable will allow to specify storage level No new revisions were added by this update. Summary of changes: python/pyspark/sql/catalog.py | 23 --- python/pyspark/sql/connect/catalog.py | 5 +++-- python/pyspark/sql/connect/plan.py | 23 +-- python/pyspark/sql/tests/test_catalog.py | 26 -- python/pyspark/sql/tests/test_dataframe.py | 16 +++- 5 files changed, 71 insertions(+), 22 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-39048][PYTHON] Refactor GroupBy._reduce_for_stat_function on accepted data types
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 30f49981ef0 [SPARK-39048][PYTHON] Refactor GroupBy._reduce_for_stat_function on accepted data types 30f49981ef0 is described below commit 30f49981ef0d71303b53f5358c021b94c6c4dc76 Author: Xinrong Meng AuthorDate: Fri Apr 29 17:56:25 2022 -0700 [SPARK-39048][PYTHON] Refactor GroupBy._reduce_for_stat_function on accepted data types ### What changes were proposed in this pull request? `Groupby._reduce_for_stat_function` is a common helper function leveraged by multiple statistical functions of GroupBy objects. It defines parameters `only_numeric` and `bool_as_numeric` to control accepted Spark types. To be consistent with pandas API, we may also have to introduce `str_as_numeric` for `sum` for example. Instead of introducing parameters designated for each Spark type, the PR is proposed to introduce a parameter `accepted_spark_types` to specify accepted types of Spark columns to be aggregated. That makes the helper function more extensible and clearer. ### Why are the changes needed? To improve code extensibility and readability. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Unit tests. Closes #36382 from xinrong-databricks/groupby.refactor_param. Authored-by: Xinrong Meng Signed-off-by: Takuya UESHIN --- python/pyspark/pandas/groupby.py | 62 +++- 1 file changed, 36 insertions(+), 26 deletions(-) diff --git a/python/pyspark/pandas/groupby.py b/python/pyspark/pandas/groupby.py index 8071f1597c3..386b24c1916 100644 --- a/python/pyspark/pandas/groupby.py +++ b/python/pyspark/pandas/groupby.py @@ -37,6 +37,7 @@ from typing import ( Sequence, Set, Tuple, +Type, Union, cast, TYPE_CHECKING, @@ -56,6 +57,7 @@ else: from pyspark.sql import Column, DataFrame as SparkDataFrame, Window, functions as F from pyspark.sql.types import ( BooleanType, +DataType, NumericType, StructField, StructType, @@ -400,7 +402,7 @@ class GroupBy(Generic[FrameLike], metaclass=ABCMeta): 1 2 3 2 2 2 """ -return self._reduce_for_stat_function(F.count, only_numeric=False) +return self._reduce_for_stat_function(F.count) # TODO: We should fix See Also when Series implementation is finished. def first(self, numeric_only: Optional[bool] = False) -> FrameLike: @@ -446,7 +448,7 @@ class GroupBy(Generic[FrameLike], metaclass=ABCMeta): 2 False 3 """ return self._reduce_for_stat_function( -F.first, only_numeric=numeric_only, bool_as_numeric=True +F.first, accepted_spark_types=(NumericType, BooleanType) if numeric_only else None ) def last(self, numeric_only: Optional[bool] = False) -> FrameLike: @@ -493,8 +495,7 @@ class GroupBy(Generic[FrameLike], metaclass=ABCMeta): """ return self._reduce_for_stat_function( lambda col: F.last(col, ignorenulls=True), -only_numeric=numeric_only, -bool_as_numeric=True, +accepted_spark_types=(NumericType, BooleanType) if numeric_only else None, ) def max(self, numeric_only: Optional[bool] = False) -> FrameLike: @@ -534,7 +535,7 @@ class GroupBy(Generic[FrameLike], metaclass=ABCMeta): 2 True 4 """ return self._reduce_for_stat_function( -F.max, only_numeric=numeric_only, bool_as_numeric=True +F.max, accepted_spark_types=(NumericType, BooleanType) if numeric_only else None ) # TODO: examples should be updated. @@ -567,7 +568,9 @@ class GroupBy(Generic[FrameLike], metaclass=ABCMeta): 1 3.0 1.33 0.33 2 4.0 1.50 1.00 """ -return self._reduce_for_stat_function(F.mean, only_numeric=True, bool_to_numeric=True) +return self._reduce_for_stat_function( +F.mean, accepted_spark_types=(NumericType,), bool_to_numeric=True +) def min(self, numeric_only: Optional[bool] = False) -> FrameLike: """ @@ -605,7 +608,7 @@ class GroupBy(Generic[FrameLike], metaclass=ABCMeta): 2 False 4 """ return self._reduce_for_stat_function( -F.min, only_numeric=numeric_only, bool_as_numeric=True +F.min, accepted_spark_types=(NumericType, BooleanType) if numeric_only else None ) # TODO: sync the doc. @@ -638,7 +641,9 @@ class GroupBy(Generic[FrameLike], metaclass=ABCMeta):
[spark] branch master updated (c00942d -> 21db916)
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git. from c00942d [SPARK-38524][SPARK-38553][K8S] Bump `Volcano` to v1.5.1 and fix Volcano `weight` to be positive integer and use `cpu` capability instead add 21db916 [SPARK-38484][PYTHON] Move usage logging instrumentation util functions from pandas module to pyspark.util module No new revisions were added by this update. Summary of changes: .../__init__.py => instrumentation_utils.py} | 244 +++-- python/pyspark/pandas/usage_logging/__init__.py| 152 + 2 files changed, 83 insertions(+), 313 deletions(-) copy python/pyspark/{pandas/usage_logging/__init__.py => instrumentation_utils.py} (55%) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated (c16a66a -> f6c4634)
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git. from c16a66a [SPARK-36194][SQL] Add a logical plan visitor to propagate the distinct attributes add f6c4634 [SPARK-37491][PYTHON] Fix Series.asof for unsorted values No new revisions were added by this update. Summary of changes: python/pyspark/pandas/series.py| 42 +- python/pyspark/pandas/tests/test_series.py | 42 ++ 2 files changed, 77 insertions(+), 7 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated (01014aa -> 0f4c26a)
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git. from 01014aa [SPARK-38486][K8S][TESTS] Upgrade the minimum Minikube version to 1.18.0 add 0f4c26a [SPARK-38387][PYTHON] Support `na_action` and Series input correspondence in `Series.map` No new revisions were added by this update. Summary of changes: python/pyspark/pandas/series.py| 36 +- python/pyspark/pandas/tests/test_series.py | 25 +++-- 2 files changed, 53 insertions(+), 8 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-37011][PYTHON] Remove unnecessary 'noqa: F401' comments
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 2b2d722 [SPARK-37011][PYTHON] Remove unnecessary 'noqa: F401' comments 2b2d722 is described below commit 2b2d7227878034331c38d6ca84ea8305b872533b Author: Takuya UESHIN AuthorDate: Tue Dec 28 12:45:20 2021 -0800 [SPARK-37011][PYTHON] Remove unnecessary 'noqa: F401' comments ### What changes were proposed in this pull request? Remove unnecessary 'noqa: F401' comments. ### Why are the changes needed? Now that `flake8` in Jenkins was upgraded (#34384), we can remove unnecessary 'noqa: F401' comments. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. Closes #34385 from ueshin/issues/SPARK-37011/F401. Authored-by: Takuya UESHIN Signed-off-by: Takuya UESHIN --- python/pyspark/pandas/_typing.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyspark/pandas/_typing.py b/python/pyspark/pandas/_typing.py index 3ab412f..51d1233 100644 --- a/python/pyspark/pandas/_typing.py +++ b/python/pyspark/pandas/_typing.py @@ -22,9 +22,9 @@ import numpy as np from pandas.api.extensions import ExtensionDtype if TYPE_CHECKING: -from pyspark.pandas.base import IndexOpsMixin # noqa: F401 (SPARK-37011) +from pyspark.pandas.base import IndexOpsMixin from pyspark.pandas.frame import DataFrame -from pyspark.pandas.generic import Frame # noqa: F401 (SPARK-37011) +from pyspark.pandas.generic import Frame from pyspark.pandas.indexes.base import Index from pyspark.pandas.series import Series - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch branch-3.2 updated: [SPARK-37678][PYTHON] Fix _cleanup_and_return signature
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch branch-3.2 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.2 by this push: new ebf7e61 [SPARK-37678][PYTHON] Fix _cleanup_and_return signature ebf7e61 is described below commit ebf7e61b90814e520e5472b9a6ad12e28f9aade3 Author: zero323 AuthorDate: Mon Dec 20 14:15:02 2021 -0800 [SPARK-37678][PYTHON] Fix _cleanup_and_return signature ### What changes were proposed in this pull request? This PR fixes return type annotation for `pandas.groupby.SeriesGroupBy._cleanup_and_return`. ### Why are the changes needed? Current annotation is incorrect (mixes pandas an pyspark.pandas types). ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. Closes #34950 from zero323/SPARK-37678. Authored-by: zero323 Signed-off-by: Takuya UESHIN (cherry picked from commit 012939077627e4f35d9585c5a46281776b770190) Signed-off-by: Takuya UESHIN --- python/pyspark/pandas/groupby.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyspark/pandas/groupby.py b/python/pyspark/pandas/groupby.py index 2c73e32..2d27b129 100644 --- a/python/pyspark/pandas/groupby.py +++ b/python/pyspark/pandas/groupby.py @@ -2967,8 +2967,8 @@ class SeriesGroupBy(GroupBy[Series]): else: return psser.copy() -def _cleanup_and_return(self, pdf: pd.DataFrame) -> Series: -return first_series(pdf).rename().rename(self._psser.name) +def _cleanup_and_return(self, psdf: DataFrame) -> Series: +return first_series(psdf).rename().rename(self._psser.name) def agg(self, *args: Any, **kwargs: Any) -> None: return MissingPandasLikeSeriesGroupBy.agg(self, *args, **kwargs) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated (eeee3f5 -> 0129390)
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git. from 3f5 [SPARK-37698][BUILD] Update ORC to 1.7.2 add 0129390 [SPARK-37678][PYTHON] Fix _cleanup_and_return signature No new revisions were added by this update. Summary of changes: python/pyspark/pandas/groupby.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated (aaf0e5e -> 8e05c78)
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git. from aaf0e5e [SPARK-37292][SQL][FOLLOWUP] Simplify the condition when removing outer join if it only has DISTINCT on streamed side add 8e05c78 [SPARK-37298][SQL] Use unique exprIds in RewriteAsOfJoin No new revisions were added by this update. Summary of changes: .../spark/sql/catalyst/optimizer/RewriteAsOfJoin.scala | 13 +++-- 1 file changed, 7 insertions(+), 6 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-37048][PYTHON] Clean up inlining type hints under SQL module
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 64eabb6 [SPARK-37048][PYTHON] Clean up inlining type hints under SQL module 64eabb6 is described below commit 64eabb6292baaaf18ee4e31cb48b204ef64aa488 Author: Takuya UESHIN AuthorDate: Wed Oct 20 16:17:01 2021 -0700 [SPARK-37048][PYTHON] Clean up inlining type hints under SQL module ### What changes were proposed in this pull request? Cleans up inlining type hints under SQL module. ### Why are the changes needed? Now that most of type hits under the SQL module are inlined, we should clean up for the module now. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? `lint-python` and existing tests should pass. Closes #34318 from ueshin/issues/SPARK-37048/cleanup. Authored-by: Takuya UESHIN Signed-off-by: Takuya UESHIN --- python/pyspark/pandas/data_type_ops/base.py | 2 +- python/pyspark/pandas/frame.py | 2 +- python/pyspark/pandas/generic.py| 4 ++-- python/pyspark/pandas/spark/functions.py| 2 +- python/pyspark/pandas/window.py | 21 ++--- python/pyspark/sql/avro/functions.py| 2 +- python/pyspark/sql/catalog.py | 10 +- python/pyspark/sql/column.py| 2 +- python/pyspark/sql/conf.py | 2 +- python/pyspark/sql/context.py | 9 - python/pyspark/sql/dataframe.py | 28 +--- python/pyspark/sql/functions.py | 4 ++-- python/pyspark/sql/group.py | 13 +++-- python/pyspark/sql/observation.py | 18 -- python/pyspark/sql/pandas/conversion.py | 6 +++--- python/pyspark/sql/pandas/group_ops.py | 4 ++-- python/pyspark/sql/readwriter.py| 27 +++ python/pyspark/sql/session.py | 16 python/pyspark/sql/streaming.py | 24 +++- python/pyspark/sql/tests/test_functions.py | 2 +- python/pyspark/sql/types.py | 20 +--- python/pyspark/sql/udf.py | 6 +++--- python/pyspark/sql/utils.py | 8 +--- python/pyspark/sql/window.py| 2 +- 24 files changed, 99 insertions(+), 135 deletions(-) diff --git a/python/pyspark/pandas/data_type_ops/base.py b/python/pyspark/pandas/data_type_ops/base.py index 47a6671..9a26d18 100644 --- a/python/pyspark/pandas/data_type_ops/base.py +++ b/python/pyspark/pandas/data_type_ops/base.py @@ -395,7 +395,7 @@ class DataTypeOps(object, metaclass=ABCMeta): collected_structed_scol = F.collect_list(structed_scol) # Sort the array by NATURAL_ORDER_COLUMN so that we can guarantee the order. collected_structed_scol = F.array_sort(collected_structed_scol) -right_values_scol = F.array([F.lit(x) for x in right]) # type: ignore +right_values_scol = F.array(*(F.lit(x) for x in right)) index_scol_names = left._internal.index_spark_column_names scol_name = left._internal.spark_column_name_for(left._internal.column_labels[0]) # Compare the values of left and right by using zip_with function. diff --git a/python/pyspark/pandas/frame.py b/python/pyspark/pandas/frame.py index c22e077..1eb91c3 100644 --- a/python/pyspark/pandas/frame.py +++ b/python/pyspark/pandas/frame.py @@ -6415,7 +6415,7 @@ defaultdict(, {'col..., 'col...})] 4 1 True 1.0 5 2 False 2.0 """ -from pyspark.sql.types import _parse_datatype_string # type: ignore[attr-defined] +from pyspark.sql.types import _parse_datatype_string include_list: List[str] if not is_list_like(include): diff --git a/python/pyspark/pandas/generic.py b/python/pyspark/pandas/generic.py index 6d9379e..03019ac 100644 --- a/python/pyspark/pandas/generic.py +++ b/python/pyspark/pandas/generic.py @@ -874,7 +874,7 @@ class Frame(object, metaclass=ABCMeta): builder = sdf.write.mode(mode) if partition_cols is not None: builder.partitionBy(partition_cols) -builder._set_opts( # type: ignore[attr-defined] +builder._set_opts( sep=sep, nullValue=na_rep, header=header, @@ -1022,7 +1022,7 @@ class Frame(object, metaclass=ABCMeta): builder = sdf.write.mode(mode) if partition_cols is not None: builder.partitionBy(partition_cols) -builder._set_opts(compression=compression) # type: ignore[attr-defined] +builder._set_opt
[spark] branch master updated (1ef6c13 -> c2ba498)
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git. from 1ef6c13 [SPARK-36933][CORE] Clean up TaskMemoryManager.acquireExecutionMemory() add c2ba498 [SPARK-36945][PYTHON] Inline type hints for python/pyspark/sql/udf.py No new revisions were added by this update. Summary of changes: python/pyspark/sql/functions.py | 2 +- python/pyspark/sql/udf.py | 129 python/pyspark/sql/udf.pyi | 58 -- 3 files changed, 91 insertions(+), 98 deletions(-) delete mode 100644 python/pyspark/sql/udf.pyi - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated (21fa3ce -> 25fc495)
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git. from 21fa3ce [SPARK-35925][SQL] Support DayTimeIntervalType in width-bucket function add 25fc495 [SPARK-36886][PYTHON] Inline type hints for python/pyspark/sql/context.py No new revisions were added by this update. Summary of changes: python/pyspark/sql/context.py | 221 +- python/pyspark/sql/context.pyi| 140 python/pyspark/sql/dataframe.py | 2 +- python/pyspark/sql/observation.py | 3 +- 4 files changed, 176 insertions(+), 190 deletions(-) delete mode 100644 python/pyspark/sql/context.pyi - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-36910][PYTHON] Inline type hints for python/pyspark/sql/types.py
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 722ac1b [SPARK-36910][PYTHON] Inline type hints for python/pyspark/sql/types.py 722ac1b is described below commit 722ac1b8b7f86fdeedf20cc11c7f547e7038029c Author: Xinrong Meng AuthorDate: Fri Oct 15 12:07:17 2021 -0700 [SPARK-36910][PYTHON] Inline type hints for python/pyspark/sql/types.py ### What changes were proposed in this pull request? Inline type hints for python/pyspark/sql/types.py ### Why are the changes needed? Current stub files cannot support type checking for the function body. Inline type hints can type check the function body. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. Closes #34174 from xinrong-databricks/inline_types. Authored-by: Xinrong Meng Signed-off-by: Takuya UESHIN --- python/pyspark/pandas/frame.py | 6 +- python/pyspark/sql/dataframe.py | 3 +- python/pyspark/sql/session.py | 2 +- python/pyspark/sql/types.py | 437 +--- python/pyspark/sql/types.pyi| 210 --- 5 files changed, 280 insertions(+), 378 deletions(-) diff --git a/python/pyspark/pandas/frame.py b/python/pyspark/pandas/frame.py index 1f9a8d0..c22e077 100644 --- a/python/pyspark/pandas/frame.py +++ b/python/pyspark/pandas/frame.py @@ -6417,12 +6417,14 @@ defaultdict(, {'col..., 'col...})] """ from pyspark.sql.types import _parse_datatype_string # type: ignore[attr-defined] +include_list: List[str] if not is_list_like(include): -include_list = [include] if include is not None else [] +include_list = [cast(str, include)] if include is not None else [] else: include_list = list(include) +exclude_list: List[str] if not is_list_like(exclude): -exclude_list = [exclude] if exclude is not None else [] +exclude_list = [cast(str, exclude)] if exclude is not None else [] else: exclude_list = list(exclude) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 223f041..7521ade 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -310,7 +310,8 @@ class DataFrame(PandasMapOpsMixin, PandasConversionMixin): """ if self._schema is None: try: -self._schema = _parse_datatype_json_string(self._jdf.schema().json()) +self._schema = cast( +StructType, _parse_datatype_json_string(self._jdf.schema().json())) except Exception as e: raise ValueError( "Unable to parse datatype from schema. %s" % e) from e diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index 60e2d69..c8ed108 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -790,7 +790,7 @@ class SparkSession(SparkConversionMixin): raise TypeError("data is already a DataFrame") if isinstance(schema, str): -schema = _parse_datatype_string(schema) +schema = cast(Union[AtomicType, StructType, str], _parse_datatype_string(schema)) elif isinstance(schema, (list, tuple)): # Must re-encode any unicode strings to be consistent with StructField names schema = [x.encode('utf-8') if not isinstance(x, str) else x for x in schema] diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index ba31fc2..69ec96e 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -25,13 +25,30 @@ import re import base64 from array import array import ctypes +from collections.abc import Iterable +from typing import ( +cast, +overload, +Any, +Callable, +Dict, +Iterator, +List, +Optional, +Union, +Tuple, +Type, +TypeVar, +) from py4j.protocol import register_input_converter -from py4j.java_gateway import JavaClass +from py4j.java_gateway import JavaClass, JavaGateway, JavaObject -from pyspark import SparkContext from pyspark.serializers import CloudPickleSerializer +T = TypeVar("T") +U = TypeVar("U") + __all__ = [ "DataType", "NullType", "StringType", "BinaryType", "BooleanType", "DateType", "TimestampType", "TimestampNTZType", "DecimalType", "DoubleType", "FloatType", @@ -42,34 +59,34 @@ __all__ = [ class DataType(object): """Base class for data types."&q
[spark] branch master updated: [SPARK-36991][PYTHON] Inline type hints for spark/python/pyspark/sql/streaming.py
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new e3b4d1e [SPARK-36991][PYTHON] Inline type hints for spark/python/pyspark/sql/streaming.py e3b4d1e is described below commit e3b4d1eef37704f8ebb5bc669ad041166cc674f0 Author: Xinrong Meng AuthorDate: Fri Oct 15 12:05:08 2021 -0700 [SPARK-36991][PYTHON] Inline type hints for spark/python/pyspark/sql/streaming.py ### What changes were proposed in this pull request? Inline type hints for spark/python/pyspark/sql/streaming.py ### Why are the changes needed? Current stub files cannot support type checking within function bodies. So we wanted to inline type hints for spark/python/pyspark/sql/streaming.py. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing test. Closes #34277 from xinrong-databricks/inline_streaming. Authored-by: Xinrong Meng Signed-off-by: Takuya UESHIN --- python/pyspark/sql/streaming.py | 355 +++ python/pyspark/sql/streaming.pyi | 197 -- 2 files changed, 250 insertions(+), 302 deletions(-) diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index 08c8934..24cd2db 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -17,15 +17,22 @@ import sys import json +from collections.abc import Iterator +from typing import cast, overload, Any, Callable, Dict, List, Optional, TYPE_CHECKING, Union -from py4j.java_gateway import java_import +from py4j.java_gateway import java_import, JavaObject -from pyspark import since, keyword_only +from pyspark import since from pyspark.sql.column import _to_seq -from pyspark.sql.readwriter import OptionUtils, to_str -from pyspark.sql.types import StructType, StructField, StringType +from pyspark.sql.readwriter import OptionUtils, to_str # type: ignore[attr-defined] +from pyspark.sql.types import Row, StructType, StructField, StringType from pyspark.sql.utils import ForeachBatchFunction, StreamingQueryException +if TYPE_CHECKING: +from pyspark.sql import SQLContext +from pyspark.sql._typing import SupportsProcess, OptionalPrimitiveType +from pyspark.sql.dataframe import DataFrame + __all__ = ["StreamingQuery", "StreamingQueryManager", "DataStreamReader", "DataStreamWriter"] @@ -41,12 +48,12 @@ class StreamingQuery(object): This API is evolving. """ -def __init__(self, jsq): +def __init__(self, jsq: JavaObject) -> None: self._jsq = jsq -@property +@property # type: ignore[misc] @since(2.0) -def id(self): +def id(self) -> str: """Returns the unique id of this query that persists across restarts from checkpoint data. That is, this id is generated when a query is started for the first time, and will be the same every time it is restarted from checkpoint data. @@ -55,17 +62,17 @@ class StreamingQuery(object): """ return self._jsq.id().toString() -@property +@property # type: ignore[misc] @since(2.1) -def runId(self): +def runId(self) -> str: """Returns the unique id of this query that does not persist across restarts. That is, every query that is started (or restarted from checkpoint) will have a different runId. """ return self._jsq.runId().toString() -@property +@property # type: ignore[misc] @since(2.0) -def name(self): +def name(self) -> str: """Returns the user-specified name of the query, or null if not specified. This name can be specified in the `org.apache.spark.sql.streaming.DataStreamWriter` as `dataframe.writeStream.queryName("query").start()`. @@ -73,15 +80,15 @@ class StreamingQuery(object): """ return self._jsq.name() -@property +@property # type: ignore[misc] @since(2.0) -def isActive(self): +def isActive(self) -> bool: """Whether this streaming query is currently active or not. """ return self._jsq.isActive() @since(2.0) -def awaitTermination(self, timeout=None): +def awaitTermination(self, timeout: Optional[int] = None) -> Optional[bool]: """Waits for the termination of `this` query, either by :func:`query.stop()` or by an exception. If the query has terminated with an exception, then the exception will be thrown. If `timeout` is set, it returns whether the query has terminated or not
[spark] branch master updated (c4e975e -> e7faa85)
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git. from c4e975e [SPARK-23626][CORE] Eagerly compute RDD.partitions on entire DAG when submitting job to DAGScheduler add e7faa85 [SPARK-36942][PYTHON] Inline type hints for python/pyspark/sql/readwriter.py No new revisions were added by this update. Summary of changes: python/pyspark/sql/readwriter.py | 474 ++ python/pyspark/sql/readwriter.pyi | 257 - 2 files changed, 373 insertions(+), 358 deletions(-) delete mode 100644 python/pyspark/sql/readwriter.pyi - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated (bb83f27 -> 2267d7f)
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git. from bb83f27 [SPARK-36938][PYTHON] Inline type hints for group.py in python/pyspark/sql add 2267d7f [SPARK-37000][PYTHON] Add type hints to python/pyspark/sql/util.py No new revisions were added by this update. Summary of changes: python/mypy.ini | 12 --- python/pyspark/sql/functions.py | 2 +- python/pyspark/sql/utils.py | 80 - 3 files changed, 49 insertions(+), 45 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-36938][PYTHON] Inline type hints for group.py in python/pyspark/sql
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new bb83f27 [SPARK-36938][PYTHON] Inline type hints for group.py in python/pyspark/sql bb83f27 is described below commit bb83f27ea7f85de9c02ee7ff6fa5b2c24efc9584 Author: dch nguyen AuthorDate: Thu Oct 14 11:15:32 2021 -0700 [SPARK-36938][PYTHON] Inline type hints for group.py in python/pyspark/sql ### What changes were proposed in this pull request? Inline type hints for python/pyspark/sql/group.py from Inline type hints for python/pyspark/sql/group.pyi. ### Why are the changes needed? Currently, there is type hint stub files python/pyspark/sql/group.pyi to show the expected types for functions, but we can also take advantage of static type checking within the functions by inlining the type hints. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existed test. Closes #34197 from dchvn/SPARK-36938. Authored-by: dch nguyen Signed-off-by: Takuya UESHIN --- python/pyspark/sql/group.py | 61 ++-- python/pyspark/sql/group.pyi | 44 2 files changed, 42 insertions(+), 63 deletions(-) diff --git a/python/pyspark/sql/group.py b/python/pyspark/sql/group.py index ab0566e..183041f 100644 --- a/python/pyspark/sql/group.py +++ b/python/pyspark/sql/group.py @@ -17,16 +17,24 @@ import sys -from pyspark.sql.column import Column, _to_seq +from typing import Callable, List, Optional, TYPE_CHECKING, overload, Dict, Union, cast, Tuple + +from py4j.java_gateway import JavaObject # type: ignore[import] + +from pyspark.sql.column import Column, _to_seq # type: ignore[attr-defined] +from pyspark.sql.context import SQLContext from pyspark.sql.dataframe import DataFrame from pyspark.sql.pandas.group_ops import PandasGroupedOpsMixin from pyspark.sql.types import StructType, StructField, IntegerType, StringType +if TYPE_CHECKING: +from pyspark.sql._typing import LiteralType + __all__ = ["GroupedData"] -def dfapi(f): -def _api(self): +def dfapi(f: Callable) -> Callable: +def _api(self: "GroupedData") -> DataFrame: name = f.__name__ jdf = getattr(self._jgd, name)() return DataFrame(jdf, self.sql_ctx) @@ -35,10 +43,13 @@ def dfapi(f): return _api -def df_varargs_api(f): -def _api(self, *cols): +def df_varargs_api(f: Callable) -> Callable: +def _api(self: "GroupedData", *cols: str) -> DataFrame: name = f.__name__ -jdf = getattr(self._jgd, name)(_to_seq(self.sql_ctx._sc, cols)) +# TODO: ignore[attr-defined] will be removed, once SparkContext is inlined +jdf = getattr(self._jgd, name)( +_to_seq(self.sql_ctx._sc, cols) # type: ignore[attr-defined] +) return DataFrame(jdf, self.sql_ctx) _api.__name__ = f.__name__ _api.__doc__ = f.__doc__ @@ -53,12 +64,20 @@ class GroupedData(PandasGroupedOpsMixin): .. versionadded:: 1.3 """ -def __init__(self, jgd, df): +def __init__(self, jgd: JavaObject, df: DataFrame): self._jgd = jgd self._df = df -self.sql_ctx = df.sql_ctx +self.sql_ctx: SQLContext = df.sql_ctx + +@overload +def agg(self, *exprs: Column) -> DataFrame: +... + +@overload +def agg(self, __exprs: Dict[str, str]) -> DataFrame: +... -def agg(self, *exprs): +def agg(self, *exprs: Union[Column, Dict[str, str]]) -> DataFrame: """Compute aggregates and returns the result as a :class:`DataFrame`. The available aggregate functions can be: @@ -115,12 +134,16 @@ class GroupedData(PandasGroupedOpsMixin): else: # Columns assert all(isinstance(c, Column) for c in exprs), "all exprs should be Column" -jdf = self._jgd.agg(exprs[0]._jc, -_to_seq(self.sql_ctx._sc, [c._jc for c in exprs[1:]])) +exprs = cast(Tuple[Column, ...], exprs) +# TODO: ignore[attr-defined] will be removed, once SparkContext is inlined +jdf = self._jgd.agg( +exprs[0]._jc, +_to_seq(self.sql_ctx._sc, [c._jc for c in exprs[1:]]) # type: ignore[attr-defined] +) return DataFrame(jdf, self.sql_ctx) @dfapi -def count(self): +def count(self) -> DataFrame: """Counts the number of records for each group. .. versionadded:: 1.3.0 @@ -132,7 +155,7 @@ class GroupedData(PandasGroupedOpsMixin): """ @df_varargs_api -def mean(self, *cols): +def mean(self, *
[spark] branch master updated: [SPARK-36951][PYTHON] Inline type hints for python/pyspark/sql/column.py
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 3ba57f5 [SPARK-36951][PYTHON] Inline type hints for python/pyspark/sql/column.py 3ba57f5 is described below commit 3ba57f5edc5594ee676249cd309b8f0d8248462e Author: Xinrong Meng AuthorDate: Tue Oct 12 13:36:22 2021 -0700 [SPARK-36951][PYTHON] Inline type hints for python/pyspark/sql/column.py ### What changes were proposed in this pull request? Inline type hints for python/pyspark/sql/column.py ### Why are the changes needed? Currently, Inline type hints for python/pyspark/sql/column.pyi doesn't support type checking within function bodies. So we inline type hints to support that. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing test. Closes #34226 from xinrong-databricks/inline_column. Authored-by: Xinrong Meng Signed-off-by: Takuya UESHIN --- python/pyspark/sql/column.py | 236 -- python/pyspark/sql/column.pyi | 118 --- python/pyspark/sql/dataframe.py | 12 +- python/pyspark/sql/functions.py | 3 +- python/pyspark/sql/observation.py | 5 +- python/pyspark/sql/window.py | 4 +- 6 files changed, 190 insertions(+), 188 deletions(-) diff --git a/python/pyspark/sql/column.py b/python/pyspark/sql/column.py index c46b0eb..a3e3e9e 100644 --- a/python/pyspark/sql/column.py +++ b/python/pyspark/sql/column.py @@ -18,25 +18,43 @@ import sys import json import warnings +from typing import ( +cast, +overload, +Any, +Callable, +Iterable, +List, +Optional, +Tuple, +TYPE_CHECKING, +Union +) + +from py4j.java_gateway import JavaObject from pyspark import copy_func from pyspark.context import SparkContext from pyspark.sql.types import DataType, StructField, StructType, IntegerType, StringType +if TYPE_CHECKING: +from pyspark.sql._typing import ColumnOrName, LiteralType, DecimalLiteral, DateTimeLiteral +from pyspark.sql.window import WindowSpec + __all__ = ["Column"] -def _create_column_from_literal(literal): -sc = SparkContext._active_spark_context +def _create_column_from_literal(literal: Union["LiteralType", "DecimalLiteral"]) -> "Column": +sc = SparkContext._active_spark_context # type: ignore[attr-defined] return sc._jvm.functions.lit(literal) -def _create_column_from_name(name): -sc = SparkContext._active_spark_context +def _create_column_from_name(name: str) -> "Column": +sc = SparkContext._active_spark_context # type: ignore[attr-defined] return sc._jvm.functions.col(name) -def _to_java_column(col): +def _to_java_column(col: "ColumnOrName") -> JavaObject: if isinstance(col, Column): jcol = col._jc elif isinstance(col, str): @@ -50,7 +68,11 @@ def _to_java_column(col): return jcol -def _to_seq(sc, cols, converter=None): +def _to_seq( +sc: SparkContext, +cols: Iterable["ColumnOrName"], +converter: Optional[Callable[["ColumnOrName"], JavaObject]] = None, +) -> JavaObject: """ Convert a list of Column (or names) into a JVM Seq of Column. @@ -59,10 +81,14 @@ def _to_seq(sc, cols, converter=None): """ if converter: cols = [converter(c) for c in cols] -return sc._jvm.PythonUtils.toSeq(cols) +return sc._jvm.PythonUtils.toSeq(cols) # type: ignore[attr-defined] -def _to_list(sc, cols, converter=None): +def _to_list( +sc: SparkContext, +cols: List["ColumnOrName"], +converter: Optional[Callable[["ColumnOrName"], JavaObject]] = None, +) -> JavaObject: """ Convert a list of Column (or names) into a JVM (Scala) List of Column. @@ -71,30 +97,37 @@ def _to_list(sc, cols, converter=None): """ if converter: cols = [converter(c) for c in cols] -return sc._jvm.PythonUtils.toList(cols) +return sc._jvm.PythonUtils.toList(cols) # type: ignore[attr-defined] -def _unary_op(name, doc="unary operator"): +def _unary_op( +name: str, +doc: str = "unary operator", +) -> Callable[["Column"], "Column"]: """ Create a method for given unary operator """ -def _(self): +def _(self: "Column") -> "Column": jc = getattr(self._jc, name)() return Column(jc) _.__doc__ = doc return _ -def _func_op(name, doc=''): -def _(self): -sc = SparkContext._active_spark_context +def _func_op(name: str, doc: str = '') -&g
[spark] branch master updated: [SPARK-36940][PYTHON] Inline type hints for python/pyspark/sql/avro/functions.py
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 255d86f [SPARK-36940][PYTHON] Inline type hints for python/pyspark/sql/avro/functions.py 255d86f is described below commit 255d86f854773c8134dc2c12e1cdcfe12aed55c1 Author: Xinrong Meng AuthorDate: Thu Oct 7 11:20:28 2021 -0700 [SPARK-36940][PYTHON] Inline type hints for python/pyspark/sql/avro/functions.py ### What changes were proposed in this pull request? Inline type hints for python/pyspark/sql/avro/functions.py. ### Why are the changes needed? Currently, we use stub files for type annotations, which don't support type checks within function bodies. So we propose to inline the type hints to support that. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. Closes #34200 from xinrong-databricks/inline_avro_func. Authored-by: Xinrong Meng Signed-off-by: Takuya UESHIN --- python/pyspark/sql/avro/functions.py | 20 +--- python/pyspark/sql/avro/functions.pyi | 27 --- 2 files changed, 13 insertions(+), 34 deletions(-) diff --git a/python/pyspark/sql/avro/functions.py b/python/pyspark/sql/avro/functions.py index dbb8ab4..27d2887 100644 --- a/python/pyspark/sql/avro/functions.py +++ b/python/pyspark/sql/avro/functions.py @@ -20,12 +20,18 @@ A collections of builtin avro functions """ +from typing import Dict, Optional, TYPE_CHECKING from pyspark import SparkContext -from pyspark.sql.column import Column, _to_java_column -from pyspark.util import _print_missing_jar +from pyspark.sql.column import Column, _to_java_column # type: ignore[attr-defined] +from pyspark.util import _print_missing_jar # type: ignore[attr-defined] +if TYPE_CHECKING: +from pyspark.sql._typing import ColumnOrName -def from_avro(data, jsonFormatSchema, options=None): + +def from_avro( +data: "ColumnOrName", jsonFormatSchema: str, options: Optional[Dict[str, str]] = None +) -> Column: """ Converts a binary column of Avro format into its corresponding catalyst value. The specified schema must match the read data, otherwise the behavior is undefined: @@ -67,7 +73,7 @@ def from_avro(data, jsonFormatSchema, options=None): [Row(value=Row(avro=Row(age=2, name='Alice')))] """ -sc = SparkContext._active_spark_context +sc = SparkContext._active_spark_context # type: ignore[attr-defined] try: jc = sc._jvm.org.apache.spark.sql.avro.functions.from_avro( _to_java_column(data), jsonFormatSchema, options or {}) @@ -78,7 +84,7 @@ def from_avro(data, jsonFormatSchema, options=None): return Column(jc) -def to_avro(data, jsonFormatSchema=""): +def to_avro(data: "ColumnOrName", jsonFormatSchema: str = "") -> Column: """ Converts a column into binary of avro format. @@ -111,7 +117,7 @@ def to_avro(data, jsonFormatSchema=""): [Row(suite=bytearray(b'\\x02\\x00'))] """ -sc = SparkContext._active_spark_context +sc = SparkContext._active_spark_context # type: ignore[attr-defined] try: if jsonFormatSchema == "": jc = sc._jvm.org.apache.spark.sql.avro.functions.to_avro(_to_java_column(data)) @@ -125,7 +131,7 @@ def to_avro(data, jsonFormatSchema=""): return Column(jc) -def _test(): +def _test() -> None: import os import sys from pyspark.testing.utils import search_jar diff --git a/python/pyspark/sql/avro/functions.pyi b/python/pyspark/sql/avro/functions.pyi deleted file mode 100644 index 4988133..000 --- a/python/pyspark/sql/avro/functions.pyi +++ /dev/null @@ -1,27 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. - -from typing import Dict, Optional - -from pyspark.sql._typing import ColumnOrName -from pyspark.sql.c
[spark] branch master updated: [SPARK-36906][PYTHON] Inline type hints for conf.py and observation.py in python/pyspark/sql
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 8181260 [SPARK-36906][PYTHON] Inline type hints for conf.py and observation.py in python/pyspark/sql 8181260 is described below commit 81812606cc32a41863c86695c4710b0e914b7e3c Author: Xinrong Meng AuthorDate: Mon Oct 4 14:32:25 2021 -0700 [SPARK-36906][PYTHON] Inline type hints for conf.py and observation.py in python/pyspark/sql ### What changes were proposed in this pull request? Inline type hints for conf.py and observation.py in python/pyspark/sql. ### Why are the changes needed? Currently, there is type hint stub files (*.pyi) to show the expected types for functions, but we can also take advantage of static type checking within the functions by inlining the type hints. ### Does this PR introduce _any_ user-facing change? No. It has a DOC typo fix: `Metrics are aggregation expressions, which are applied to the DataFrame while **is** is being` is changed to `Metrics are aggregation expressions, which are applied to the DataFrame while **it** is being` ### How was this patch tested? Existing test. Closes #34159 from xinrong-databricks/inline_conf_observation. Authored-by: Xinrong Meng Signed-off-by: Takuya UESHIN --- python/pyspark/__init__.pyi| 4 ++-- python/pyspark/sql/conf.py | 19 +++ python/pyspark/sql/conf.pyi| 27 --- python/pyspark/sql/observation.py | 24 ++-- python/pyspark/sql/observation.pyi | 27 --- 5 files changed, 27 insertions(+), 74 deletions(-) diff --git a/python/pyspark/__init__.pyi b/python/pyspark/__init__.pyi index 07cbccb..f85319b 100644 --- a/python/pyspark/__init__.pyi +++ b/python/pyspark/__init__.pyi @@ -16,7 +16,7 @@ # specific language governing permissions and limitations # under the License. -from typing import Callable, Optional, TypeVar +from typing import Callable, Optional, TypeVar, Union from pyspark.accumulators import ( # noqa: F401 Accumulator as Accumulator, @@ -67,7 +67,7 @@ from pyspark.sql import ( # noqa: F401 T = TypeVar("T") F = TypeVar("F", bound=Callable) -def since(version: str) -> Callable[[T], T]: ... +def since(version: Union[str, float]) -> Callable[[T], T]: ... def copy_func( f: F, name: Optional[str] = ..., diff --git a/python/pyspark/sql/conf.py b/python/pyspark/sql/conf.py index eab084a..54ae6fb 100644 --- a/python/pyspark/sql/conf.py +++ b/python/pyspark/sql/conf.py @@ -16,8 +16,11 @@ # import sys +from typing import Any, Optional -from pyspark import since, _NoValue +from py4j.java_gateway import JavaObject + +from pyspark import since, _NoValue # type: ignore[attr-defined] class RuntimeConfig(object): @@ -26,17 +29,17 @@ class RuntimeConfig(object): Options set here are automatically propagated to the Hadoop configuration during I/O. """ -def __init__(self, jconf): +def __init__(self, jconf: JavaObject) -> None: """Create a new RuntimeConfig that wraps the underlying JVM object.""" self._jconf = jconf @since(2.0) -def set(self, key, value): +def set(self, key: str, value: str) -> None: """Sets the given Spark runtime configuration property.""" self._jconf.set(key, value) @since(2.0) -def get(self, key, default=_NoValue): +def get(self, key: str, default: Optional[str] = _NoValue) -> str: """Returns the value of Spark runtime configuration property for the given key, assuming it is set. """ @@ -49,25 +52,25 @@ class RuntimeConfig(object): return self._jconf.get(key, default) @since(2.0) -def unset(self, key): +def unset(self, key: str) -> None: """Resets the configuration property for the given key.""" self._jconf.unset(key) -def _checkType(self, obj, identifier): +def _checkType(self, obj: Any, identifier: str) -> None: """Assert that an object is of type str.""" if not isinstance(obj, str): raise TypeError("expected %s '%s' to be a string (was '%s')" % (identifier, obj, type(obj).__name__)) @since(2.4) -def isModifiable(self, key): +def isModifiable(self, key: str) -> bool: """Indicates whether the configuration property with the given key is modifiable in the current session. """ return self
[spark] branch branch-3.2 updated: [SPARK-36818][PYTHON] Fix filtering a Series by a boolean Series
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch branch-3.2 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.2 by this push: new 423cff4 [SPARK-36818][PYTHON] Fix filtering a Series by a boolean Series 423cff4 is described below commit 423cff45672685393b4912e318839b1ff428b72b Author: Xinrong Meng AuthorDate: Wed Sep 22 12:52:56 2021 -0700 [SPARK-36818][PYTHON] Fix filtering a Series by a boolean Series ### What changes were proposed in this pull request? Fix filtering a Series (without a name) by a boolean Series. ### Why are the changes needed? A bugfix. The issue is raised as https://github.com/databricks/koalas/issues/2199. ### Does this PR introduce _any_ user-facing change? Yes. From ```py >>> psser = ps.Series([0, 1, 2, 3, 4]) >>> ps.set_option('compute.ops_on_diff_frames', True) >>> psser.loc[ps.Series([True, True, True, False, False])] Traceback (most recent call last): ... KeyError: 'none key' ``` To ```py >>> psser = ps.Series([0, 1, 2, 3, 4]) >>> ps.set_option('compute.ops_on_diff_frames', True) >>> psser.loc[ps.Series([True, True, True, False, False])] 00 11 22 dtype: int64 ``` ### How was this patch tested? Unit test. Closes #34061 from xinrong-databricks/filter_series. Authored-by: Xinrong Meng Signed-off-by: Takuya UESHIN (cherry picked from commit 6a5ee0283cee29a965e393fd829ff3cd0b09cf4d) Signed-off-by: Takuya UESHIN --- python/pyspark/pandas/indexing.py | 6 -- python/pyspark/pandas/tests/test_indexing.py | 9 + python/pyspark/pandas/tests/test_ops_on_diff_frames.py | 6 ++ 3 files changed, 19 insertions(+), 2 deletions(-) diff --git a/python/pyspark/pandas/indexing.py b/python/pyspark/pandas/indexing.py index bf74f06..125964c 100644 --- a/python/pyspark/pandas/indexing.py +++ b/python/pyspark/pandas/indexing.py @@ -33,6 +33,7 @@ import numpy as np from pyspark import pandas as ps # noqa: F401 from pyspark.pandas._typing import Label, Name, Scalar from pyspark.pandas.internal import ( +DEFAULT_SERIES_NAME, InternalField, InternalFrame, NATURAL_ORDER_COLUMN_NAME, @@ -435,11 +436,12 @@ class LocIndexerLike(IndexerLike, metaclass=ABCMeta): if self._is_series: if isinstance(key, Series) and not same_anchor(key, self._psdf_or_psser): -psdf = self._psdf_or_psser.to_frame() +name = self._psdf_or_psser.name or DEFAULT_SERIES_NAME +psdf = self._psdf_or_psser.to_frame(name) temp_col = verify_temp_column_name(psdf, "__temp_col__") psdf[temp_col] = key -return type(self)(psdf[self._psdf_or_psser.name])[psdf[temp_col]] +return type(self)(psdf[name].rename(self._psdf_or_psser.name))[psdf[temp_col]] cond, limit, remaining_index = self._select_rows(key) if cond is None and limit is None: diff --git a/python/pyspark/pandas/tests/test_indexing.py b/python/pyspark/pandas/tests/test_indexing.py index b74cf90..2b00b3f 100644 --- a/python/pyspark/pandas/tests/test_indexing.py +++ b/python/pyspark/pandas/tests/test_indexing.py @@ -417,6 +417,15 @@ class IndexingTest(PandasOnSparkTestCase): self.assertRaises(KeyError, lambda: psdf.loc[0:30]) self.assertRaises(KeyError, lambda: psdf.loc[10:100]) +def test_loc_getitem_boolean_series(self): +pdf = pd.DataFrame( +{"A": [0, 1, 2, 3, 4], "B": [100, 200, 300, 400, 500]}, index=[20, 10, 30, 0, 50] +) +psdf = ps.from_pandas(pdf) +self.assert_eq(pdf.A.loc[pdf.B > 200], psdf.A.loc[psdf.B > 200]) +self.assert_eq(pdf.B.loc[pdf.B > 200], psdf.B.loc[psdf.B > 200]) +self.assert_eq(pdf.loc[pdf.B > 200], psdf.loc[psdf.B > 200]) + def test_loc_non_informative_index(self): pdf = pd.DataFrame({"x": [1, 2, 3, 4]}, index=[10, 20, 30, 40]) psdf = ps.from_pandas(pdf) diff --git a/python/pyspark/pandas/tests/test_ops_on_diff_frames.py b/python/pyspark/pandas/tests/test_ops_on_diff_frames.py index 11132ad..0dedfdc 100644 --- a/python/pyspark/pandas/tests/test_ops_on_diff_frames.py +++ b/python/pyspark/pandas/tests/test_ops_on_diff_frames.py @@ -503,6 +503,12 @@ class OpsOnDiffFramesEnabledTest(PandasOnSparkTestCase, SQLTestUtils): (pdf1.A + 1).loc[pdf2.A > -3].sort_index(), (psdf1.A + 1).loc[psdf2.A > -3].sort_index() ) +pser = pd.Series([0, 1, 2, 3, 4], index=[20, 10, 30, 0, 50]) +psser = ps.from_pandas(pser) +sel
[spark] branch master updated (a7cbe69 -> 6a5ee02)
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git. from a7cbe69 [SPARK-36753][SQL] ArrayExcept handle duplicated Double.NaN and Float.NaN add 6a5ee02 [SPARK-36818][PYTHON] Fix filtering a Series by a boolean Series No new revisions were added by this update. Summary of changes: python/pyspark/pandas/indexing.py | 6 -- python/pyspark/pandas/tests/test_indexing.py | 9 + python/pyspark/pandas/tests/test_ops_on_diff_frames.py | 6 ++ 3 files changed, 19 insertions(+), 2 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch branch-3.2 updated (affd7a4 -> 4543ac6)
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a change to branch branch-3.2 in repository https://gitbox.apache.org/repos/asf/spark.git. from affd7a4 [SPARK-36670][FOLLOWUP][TEST] Remove brotli-codec dependency add 4543ac6 [SPARK-36771][PYTHON][3.2] Fix `pop` of Categorical Series No new revisions were added by this update. Summary of changes: python/pyspark/pandas/series.py| 8 ++-- python/pyspark/pandas/tests/test_series.py | 25 + 2 files changed, 31 insertions(+), 2 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-36771][PYTHON] Fix `pop` of Categorical Series
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 079a9c5 [SPARK-36771][PYTHON] Fix `pop` of Categorical Series 079a9c5 is described below commit 079a9c52925818532b57c9cec1ddd31be723885e Author: Xinrong Meng AuthorDate: Tue Sep 21 14:11:21 2021 -0700 [SPARK-36771][PYTHON] Fix `pop` of Categorical Series ### What changes were proposed in this pull request? Fix `pop` of Categorical Series to be consistent with the latest pandas (1.3.2) behavior. ### Why are the changes needed? As https://github.com/databricks/koalas/issues/2198, pandas API on Spark behaves differently from pandas on `pop` of Categorical Series. ### Does this PR introduce _any_ user-facing change? Yes, results of `pop` of Categorical Series change. From ```py >>> psser = ps.Series(["a", "b", "c", "a"], dtype="category") >>> psser 0a 1b 2c 3a dtype: category Categories (3, object): ['a', 'b', 'c'] >>> psser.pop(0) 0 >>> psser 1b 2c 3a dtype: category Categories (3, object): ['a', 'b', 'c'] >>> psser.pop(3) 0 >>> psser 1b 2c dtype: category Categories (3, object): ['a', 'b', 'c'] ``` To ```py >>> psser = ps.Series(["a", "b", "c", "a"], dtype="category") >>> psser 0a 1b 2c 3a dtype: category Categories (3, object): ['a', 'b', 'c'] >>> psser.pop(0) 'a' >>> psser 1b 2c 3a dtype: category Categories (3, object): ['a', 'b', 'c'] >>> psser.pop(3) 'a' >>> psser 1b 2c dtype: category Categories (3, object): ['a', 'b', 'c'] ``` ### How was this patch tested? Unit tests. Closes #34052 from xinrong-databricks/cat_pop. Authored-by: Xinrong Meng Signed-off-by: Takuya UESHIN --- python/pyspark/pandas/series.py| 8 ++-- python/pyspark/pandas/tests/test_series.py | 25 + 2 files changed, 31 insertions(+), 2 deletions(-) diff --git a/python/pyspark/pandas/series.py b/python/pyspark/pandas/series.py index d72c08d..da0d2fb 100644 --- a/python/pyspark/pandas/series.py +++ b/python/pyspark/pandas/series.py @@ -47,7 +47,7 @@ import numpy as np import pandas as pd from pandas.core.accessor import CachedAccessor from pandas.io.formats.printing import pprint_thing -from pandas.api.types import is_list_like, is_hashable +from pandas.api.types import is_list_like, is_hashable, CategoricalDtype from pandas.tseries.frequencies import DateOffset from pyspark.sql import functions as F, Column, DataFrame as SparkDataFrame from pyspark.sql.types import ( @@ -4098,7 +4098,11 @@ class Series(Frame, IndexOpsMixin, Generic[T]): pdf = sdf.limit(2).toPandas() length = len(pdf) if length == 1: -return pdf[internal.data_spark_column_names[0]].iloc[0] +val = pdf[internal.data_spark_column_names[0]].iloc[0] +if isinstance(self.dtype, CategoricalDtype): +return self.dtype.categories[val] +else: +return val item_string = name_like_string(item) sdf = sdf.withColumn(SPARK_DEFAULT_INDEX_NAME, SF.lit(str(item_string))) diff --git a/python/pyspark/pandas/tests/test_series.py b/python/pyspark/pandas/tests/test_series.py index 09e5d30..b7bb121 100644 --- a/python/pyspark/pandas/tests/test_series.py +++ b/python/pyspark/pandas/tests/test_series.py @@ -1669,6 +1669,31 @@ class SeriesTest(PandasOnSparkTestCase, SQLTestUtils): with self.assertRaisesRegex(KeyError, msg): psser.pop(("lama", "speed", "x")) +pser = pd.Series(["a", "b", "c", "a"], dtype="category") +psser = ps.from_pandas(pser) + +if LooseVersion(pd.__version__) >= LooseVersion("1.3.0"): +self.assert_eq(psser.pop(0), pser.pop(0)) +self.assert_eq(psser, pser) + +self.assert_eq(psser.pop(3), pser.pop(3)) +self.assert_eq(psser, pser) +else: +# Before pandas 1.3.0, `pop` modifies the dtype of categorical series wrongly. +
[spark] branch master updated: [SPARK-36769][PYTHON] Improve `filter` of single-indexed DataFrame
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 33e463c [SPARK-36769][PYTHON] Improve `filter` of single-indexed DataFrame 33e463c is described below commit 33e463ccf99d09ad8a743d32104f590e204da93d Author: Xinrong Meng AuthorDate: Tue Sep 21 10:20:15 2021 -0700 [SPARK-36769][PYTHON] Improve `filter` of single-indexed DataFrame ### What changes were proposed in this pull request? Improve `filter` of single-indexed DataFrame by replacing a long Project with Filter or Join. ### Why are the changes needed? When the given `items` have too many elements, a long Project is introduced. We may replace that with `Column.isin` or joining depending on the length of `items` for better performance. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Unit tests. Closes #33998 from xinrong-databricks/impr_filter. Authored-by: Xinrong Meng Signed-off-by: Takuya UESHIN --- python/pyspark/pandas/frame.py| 28 +++ python/pyspark/pandas/tests/test_dataframe.py | 7 +++ 2 files changed, 27 insertions(+), 8 deletions(-) diff --git a/python/pyspark/pandas/frame.py b/python/pyspark/pandas/frame.py index 09efef2..cba1db1 100644 --- a/python/pyspark/pandas/frame.py +++ b/python/pyspark/pandas/frame.py @@ -9995,13 +9995,25 @@ defaultdict(, {'col..., 'col...})] raise ValueError("items should be a list-like object.") if axis == 0: if len(index_scols) == 1: -col = None -for item in items: -if col is None: -col = index_scols[0] == SF.lit(item) -else: -col = col | (index_scols[0] == SF.lit(item)) -elif len(index_scols) > 1: +if len(items) <= ps.get_option("compute.isin_limit"): +col = index_scols[0].isin([SF.lit(item) for item in items]) +return DataFrame(self._internal.with_filter(col)) +else: +item_sdf_col = verify_temp_column_name( +self._internal.spark_frame, "__item__" +) +item_sdf = default_session().createDataFrame( +pd.DataFrame({item_sdf_col: items}) +) +joined_sdf = self._internal.spark_frame.join( +other=F.broadcast(item_sdf), +on=(index_scols[0] == scol_for(item_sdf, item_sdf_col)), +how="semi", +) + +return DataFrame(self._internal.with_new_sdf(joined_sdf)) + +else: # for multi-index col = None for item in items: @@ -10019,7 +10031,7 @@ defaultdict(, {'col..., 'col...})] col = midx_col else: col = col | midx_col -return DataFrame(self._internal.with_filter(col)) +return DataFrame(self._internal.with_filter(col)) else: return self[items] elif like is not None: diff --git a/python/pyspark/pandas/tests/test_dataframe.py b/python/pyspark/pandas/tests/test_dataframe.py index 20aecc2..3cfbc03 100644 --- a/python/pyspark/pandas/tests/test_dataframe.py +++ b/python/pyspark/pandas/tests/test_dataframe.py @@ -4313,6 +4313,13 @@ class DataFrameTest(PandasOnSparkTestCase, SQLTestUtils): psdf.filter(items=["ab", "aa"], axis=0).sort_index(), pdf.filter(items=["ab", "aa"], axis=0).sort_index(), ) + +with option_context("compute.isin_limit", 0): +self.assert_eq( +psdf.filter(items=["ab", "aa"], axis=0).sort_index(), +pdf.filter(items=["ab", "aa"], axis=0).sort_index(), +) + self.assert_eq( psdf.filter(items=["ba", "db"], axis=1).sort_index(), pdf.filter(items=["ba", "db"], axis=1).sort_index(), - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch branch-3.2 updated: [SPARK-36785][PYTHON] Fix DataFrame.isin when DataFrame has NaN value
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch branch-3.2 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.2 by this push: new 3d47c69 [SPARK-36785][PYTHON] Fix DataFrame.isin when DataFrame has NaN value 3d47c69 is described below commit 3d47c692d276d2d489664aa2d4e66e23e9bae0f7 Author: dgd-contributor AuthorDate: Mon Sep 20 17:52:51 2021 -0700 [SPARK-36785][PYTHON] Fix DataFrame.isin when DataFrame has NaN value ### What changes were proposed in this pull request? Fix DataFrame.isin when DataFrame has NaN value ### Why are the changes needed? Fix DataFrame.isin when DataFrame has NaN value ``` python >>> psdf = ps.DataFrame( ... {"a": [None, 2, 3, 4, 5, 6, 7, 8, None], "b": [None, 5, None, 3, 2, 1, None, 0, 0], "c": [1, 5, 1, 3, 2, 1, 1, 0, 0]}, ... ) >>> psdf ab c 0 NaN NaN 1 1 2.0 5.0 5 2 3.0 NaN 1 3 4.0 3.0 3 4 5.0 2.0 2 5 6.0 1.0 1 6 7.0 NaN 1 7 8.0 0.0 0 8 NaN 0.0 0 >>> other = [1, 2, None] >>> psdf.isin(other) a b c 0 None None True 1 True None None 2 None None True 3 None None None 4 None True True 5 None True True 6 None None True 7 None None None 8 None None None >>> psdf.to_pandas().isin(other) a b c 0 False False True 1 True False False 2 False False True 3 False False False 4 False True True 5 False True True 6 False False True 7 False False False 8 False False False ``` ### Does this PR introduce _any_ user-facing change? After this PR ``` python >>> psdf = ps.DataFrame( ... {"a": [None, 2, 3, 4, 5, 6, 7, 8, None], "b": [None, 5, None, 3, 2, 1, None, 0, 0], "c": [1, 5, 1, 3, 2, 1, 1, 0, 0]}, ... ) >>> psdf ab c 0 NaN NaN 1 1 2.0 5.0 5 2 3.0 NaN 1 3 4.0 3.0 3 4 5.0 2.0 2 5 6.0 1.0 1 6 7.0 NaN 1 7 8.0 0.0 0 8 NaN 0.0 0 >>> other = [1, 2, None] >>> psdf.isin(other) a b c 0 False False True 1 True False False 2 False False True 3 False False False 4 False True True 5 False True True 6 False False True 7 False False False 8 False False False ``` ### How was this patch tested? Unit tests Closes #34040 from dgd-contributor/SPARK-36785_dataframe.isin_fix. Authored-by: dgd-contributor Signed-off-by: Takuya UESHIN (cherry picked from commit cc182fe6f61eab494350b81196b3cce356814a25) Signed-off-by: Takuya UESHIN --- python/pyspark/pandas/frame.py| 34 +++--- python/pyspark/pandas/tests/test_dataframe.py | 35 +++ 2 files changed, 55 insertions(+), 14 deletions(-) diff --git a/python/pyspark/pandas/frame.py b/python/pyspark/pandas/frame.py index ec6b261..e576789 100644 --- a/python/pyspark/pandas/frame.py +++ b/python/pyspark/pandas/frame.py @@ -7394,31 +7394,37 @@ defaultdict(, {'col..., 'col...})] if col in values: item = values[col] item = item.tolist() if isinstance(item, np.ndarray) else list(item) -data_spark_columns.append( - self._internal.spark_column_for(self._internal.column_labels[i]) -.isin(item) -.alias(self._internal.data_spark_column_names[i]) + +scol = self._internal.spark_column_for(self._internal.column_labels[i]).isin( +[SF.lit(v) for v in item] ) +scol = F.coalesce(scol, F.lit(False)) else: -data_spark_columns.append( - SF.lit(False).alias(self._internal.data_spark_column_names[i]) -) +scol = SF.lit(False) + data_spark_columns.append(scol.alias(self._internal.data_spark_column_names[i])) elif is_list_like(values): values = ( cast(np.ndarray, values).tolist() if isinstance(values, np.ndarray) else list(values) ) -data_spark_columns += [ -self._internal.spark_column_for(label) -.isin(values) -.alias(self._internal.spark_column_name_for(label)) -for label in self._internal.column_labels -] + +for label i
[spark] branch master updated (4b61c62 -> cc182fe)
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git. from 4b61c62 [SPARK-36746][PYTHON] Refactor `_select_rows_by_iterable` in `iLocIndexer` to use `Column.isin` add cc182fe [SPARK-36785][PYTHON] Fix DataFrame.isin when DataFrame has NaN value No new revisions were added by this update. Summary of changes: python/pyspark/pandas/frame.py| 34 +++--- python/pyspark/pandas/tests/test_dataframe.py | 35 +++ 2 files changed, 55 insertions(+), 14 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-36746][PYTHON] Refactor `_select_rows_by_iterable` in `iLocIndexer` to use `Column.isin`
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 4b61c62 [SPARK-36746][PYTHON] Refactor `_select_rows_by_iterable` in `iLocIndexer` to use `Column.isin` 4b61c62 is described below commit 4b61c623b52032c7e4749db641c5e63c1317c3a4 Author: Xinrong Meng AuthorDate: Mon Sep 20 15:00:10 2021 -0700 [SPARK-36746][PYTHON] Refactor `_select_rows_by_iterable` in `iLocIndexer` to use `Column.isin` ### What changes were proposed in this pull request? Refactor `_select_rows_by_iterable` in `iLocIndexer` to use `Column.isin`. ### Why are the changes needed? For better performance. After a rough benchmark, a long projection performs worse than `Column.isin`, even when the length of the filtering conditions exceeding `compute.isin_limit`. So we use `Column.isin` instead. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. Closes #33964 from xinrong-databricks/iloc_select. Authored-by: Xinrong Meng Signed-off-by: Takuya UESHIN --- python/pyspark/pandas/indexing.py | 15 +++ 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/python/pyspark/pandas/indexing.py b/python/pyspark/pandas/indexing.py index c46e250..3e07975 100644 --- a/python/pyspark/pandas/indexing.py +++ b/python/pyspark/pandas/indexing.py @@ -1657,14 +1657,13 @@ class iLocIndexer(LocIndexerLike): "however, normalised index was [%s]" % new_rows_sel ) -sequence_scol = sdf[self._sequence_col] -cond = [] -for key in new_rows_sel: -cond.append(sequence_scol == SF.lit(int(key)).cast(LongType())) - -if len(cond) == 0: -cond = [SF.lit(False)] -return reduce(lambda x, y: x | y, cond), None, None +if len(new_rows_sel) == 0: +cond = SF.lit(False) +else: +cond = sdf[self._sequence_col].isin( +[SF.lit(int(key)).cast(LongType()) for key in new_rows_sel] +) +return cond, None, None def _select_rows_else( self, rows_sel: Any - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-36618][PYTHON] Support dropping rows of a single-indexed DataFrame
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 4cf86d3 [SPARK-36618][PYTHON] Support dropping rows of a single-indexed DataFrame 4cf86d3 is described below commit 4cf86d33adc483382a2803486db628e21cad44e9 Author: Xinrong Meng AuthorDate: Mon Sep 20 14:50:50 2021 -0700 [SPARK-36618][PYTHON] Support dropping rows of a single-indexed DataFrame ### What changes were proposed in this pull request? Support dropping rows of a single-indexed DataFrame. Dropping rows and columns at the same time is supported in this PR as well. ### Why are the changes needed? To increase pandas API coverage. ### Does this PR introduce _any_ user-facing change? Yes, dropping rows of a single-indexed DataFrame is supported now. ```py >>> df = ps.DataFrame(np.arange(12).reshape(3, 4), columns=['A', 'B', 'C', 'D']) >>> df A B C D 0 0 1 2 3 1 4 5 6 7 2 8 9 10 11 ``` From ```py >>> df.drop([0, 1]) Traceback (most recent call last): ... KeyError: [(0,), (1,)] >>> df.drop([0, 1], axis=0) Traceback (most recent call last): ... NotImplementedError: Drop currently only works for axis=1 >>> df.drop(1) Traceback (most recent call last): ... KeyError: [(1,)] >>> df.drop(index=1) Traceback (most recent call last): ... TypeError: drop() got an unexpected keyword argument 'index' >>> df.drop(index=[0, 1], columns='A') Traceback (most recent call last): ... TypeError: drop() got an unexpected keyword argument 'index' ``` To ```py >>> df.drop([0, 1]) A B C D 2 8 9 10 11 >>> df.drop([0, 1], axis=0) A B C D 2 8 9 10 11 >>> df.drop(1) A B C D 0 0 1 2 3 2 8 9 10 11 >>> df.drop(index=1) A B C D 0 0 1 2 3 2 8 9 10 11 >>> df.drop(index=[0, 1], columns='A') B C D 2 9 10 11 ``` ### How was this patch tested? Unit tests. Closes #33929 from xinrong-databricks/frame_drop. Authored-by: Xinrong Meng Signed-off-by: Takuya UESHIN --- .../source/migration_guide/pyspark_3.2_to_3.3.rst | 23 +++ python/pyspark/pandas/frame.py | 176 ++--- python/pyspark/pandas/indexing.py | 4 +- python/pyspark/pandas/tests/test_dataframe.py | 106 +++-- python/pyspark/pandas/tests/test_groupby.py| 8 +- 5 files changed, 241 insertions(+), 76 deletions(-) diff --git a/python/docs/source/migration_guide/pyspark_3.2_to_3.3.rst b/python/docs/source/migration_guide/pyspark_3.2_to_3.3.rst new file mode 100644 index 000..060f24c --- /dev/null +++ b/python/docs/source/migration_guide/pyspark_3.2_to_3.3.rst @@ -0,0 +1,23 @@ +.. Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + +..http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. + + += +Upgrading from PySpark 3.2 to 3.3 += + +* In Spark 3.3, the ``drop`` method of pandas API on Spark DataFrame supports dropping rows by ``index``, and sets dropping by index instead of column by default. diff --git a/python/pyspark/pandas/frame.py b/python/pyspark/pandas/frame.py index 8d37d31..f863890 100644 --- a/python/pyspark/pandas/frame.py +++ b/python/pyspark/pandas/frame.py @@ -2817,7 +2817,7 @@ defaultdict(, {'col..., 'col...})] 3 NaN """ result = self[item] -self._update_internal_frame(self.drop(item)._internal) +self._update_internal_frame(self.drop(columns=item)._internal) return result # TODO: add axis parameter can work when '1' or 'column
[spark] branch branch-3.2 updated: [SPARK-36762][PYTHON] Fix Series.isin when Series has NaN values
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch branch-3.2 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.2 by this push: new 36ce9cc [SPARK-36762][PYTHON] Fix Series.isin when Series has NaN values 36ce9cc is described below commit 36ce9cce55619160db6104da10b9b53141c10105 Author: dgd-contributor AuthorDate: Fri Sep 17 17:48:15 2021 -0700 [SPARK-36762][PYTHON] Fix Series.isin when Series has NaN values ### What changes were proposed in this pull request? Fix Series.isin when Series has NaN values ### Why are the changes needed? Fix Series.isin when Series has NaN values ``` python >>> pser = pd.Series([None, 5, None, 3, 2, 1, None, 0, 0]) >>> psser = ps.from_pandas(pser) >>> pser.isin([1, 3, 5, None]) 0False 1 True 2False 3 True 4False 5 True 6False 7False 8False dtype: bool >>> psser.isin([1, 3, 5, None]) 0None 1True 2None 3True 4None 5True 6None 7None 8None dtype: object ``` ### Does this PR introduce _any_ user-facing change? After this PR ``` python >>> pser = pd.Series([None, 5, None, 3, 2, 1, None, 0, 0]) >>> psser = ps.from_pandas(pser) >>> psser.isin([1, 3, 5, None]) 0False 1 True 2False 3 True 4False 5 True 6False 7False 8False dtype: bool ``` ### How was this patch tested? unit tests Closes #34005 from dgd-contributor/SPARK-36762_fix_series.isin_when_values_have_NaN. Authored-by: dgd-contributor Signed-off-by: Takuya UESHIN (cherry picked from commit 32b8512912c211f7f12e717e7029e89645da9d3b) Signed-off-by: Takuya UESHIN --- python/pyspark/pandas/base.py | 10 -- python/pyspark/pandas/tests/test_series.py | 17 + 2 files changed, 25 insertions(+), 2 deletions(-) diff --git a/python/pyspark/pandas/base.py b/python/pyspark/pandas/base.py index 58f6c19..bdec3ac 100644 --- a/python/pyspark/pandas/base.py +++ b/python/pyspark/pandas/base.py @@ -27,7 +27,7 @@ import numpy as np import pandas as pd # noqa: F401 from pandas.api.types import is_list_like, CategoricalDtype from pyspark.sql import functions as F, Column, Window -from pyspark.sql.types import LongType +from pyspark.sql.types import LongType, BooleanType from pyspark import pandas as ps # For running doctests and reference resolution in PyCharm. from pyspark.pandas._typing import Axis, Dtype, IndexOpsLike, Label, SeriesOrIndex @@ -867,7 +867,13 @@ class IndexOpsMixin(object, metaclass=ABCMeta): ) values = values.tolist() if isinstance(values, np.ndarray) else list(values) -return self._with_new_scol(self.spark.column.isin([SF.lit(v) for v in values])) + +other = [SF.lit(v) for v in values] +scol = self.spark.column.isin(other) +field = self._internal.data_fields[0].copy( +dtype=np.dtype("bool"), spark_type=BooleanType(), nullable=False +) +return self._with_new_scol(scol=F.coalesce(scol, F.lit(False)), field=field) def isnull(self: IndexOpsLike) -> IndexOpsLike: """ diff --git a/python/pyspark/pandas/tests/test_series.py b/python/pyspark/pandas/tests/test_series.py index de5ba60..cbfc999 100644 --- a/python/pyspark/pandas/tests/test_series.py +++ b/python/pyspark/pandas/tests/test_series.py @@ -394,6 +394,23 @@ class SeriesTest(PandasOnSparkTestCase, SQLTestUtils): with self.assertRaisesRegex(TypeError, msg): psser.isin(1) +# when Series have NaN +pser = pd.Series(["lama", "cow", None, "lama", "beetle", "lama", "hippo", None], name="a") +psser = ps.from_pandas(pser) + +self.assert_eq(psser.isin(["cow", "lama"]), pser.isin(["cow", "lama"])) + +pser = pd.Series([None, 5, None, 3, 2, 1, None, 0, 0], name="a") +psser = ps.from_pandas(pser) + +if LooseVersion(pd.__version__) >= LooseVersion("1.2"): +self.assert_eq(psser.isin([1, 5, 0, None]), pser.isin([1, 5, 0, None])) +else: +expected = pd.Series( +[False, True, False, False, False, True, False, True, True], name="a" +) +self.assert_eq(psser.isin([1, 5, 0, None]), expected) + def test_drop_duplicates(self): pdf = pd.DataFrame({"animal": ["lama", "cow", "lama", "beetle", "lama", "hippo"]}) psdf = ps.from_pandas(pdf) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-36762][PYTHON] Fix Series.isin when Series has NaN values
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 32b8512 [SPARK-36762][PYTHON] Fix Series.isin when Series has NaN values 32b8512 is described below commit 32b8512912c211f7f12e717e7029e89645da9d3b Author: dgd-contributor AuthorDate: Fri Sep 17 17:48:15 2021 -0700 [SPARK-36762][PYTHON] Fix Series.isin when Series has NaN values ### What changes were proposed in this pull request? Fix Series.isin when Series has NaN values ### Why are the changes needed? Fix Series.isin when Series has NaN values ``` python >>> pser = pd.Series([None, 5, None, 3, 2, 1, None, 0, 0]) >>> psser = ps.from_pandas(pser) >>> pser.isin([1, 3, 5, None]) 0False 1 True 2False 3 True 4False 5 True 6False 7False 8False dtype: bool >>> psser.isin([1, 3, 5, None]) 0None 1True 2None 3True 4None 5True 6None 7None 8None dtype: object ``` ### Does this PR introduce _any_ user-facing change? After this PR ``` python >>> pser = pd.Series([None, 5, None, 3, 2, 1, None, 0, 0]) >>> psser = ps.from_pandas(pser) >>> psser.isin([1, 3, 5, None]) 0False 1 True 2False 3 True 4False 5 True 6False 7False 8False dtype: bool ``` ### How was this patch tested? unit tests Closes #34005 from dgd-contributor/SPARK-36762_fix_series.isin_when_values_have_NaN. Authored-by: dgd-contributor Signed-off-by: Takuya UESHIN --- python/pyspark/pandas/base.py | 10 -- python/pyspark/pandas/tests/test_series.py | 17 + 2 files changed, 25 insertions(+), 2 deletions(-) diff --git a/python/pyspark/pandas/base.py b/python/pyspark/pandas/base.py index 533460c..27f3d78 100644 --- a/python/pyspark/pandas/base.py +++ b/python/pyspark/pandas/base.py @@ -27,7 +27,7 @@ import numpy as np import pandas as pd # noqa: F401 from pandas.api.types import is_list_like, CategoricalDtype from pyspark.sql import functions as F, Column, Window -from pyspark.sql.types import LongType +from pyspark.sql.types import LongType, BooleanType from pyspark import pandas as ps # For running doctests and reference resolution in PyCharm. from pyspark.pandas._typing import Axis, Dtype, IndexOpsLike, Label, SeriesOrIndex @@ -873,7 +873,13 @@ class IndexOpsMixin(object, metaclass=ABCMeta): ) values = values.tolist() if isinstance(values, np.ndarray) else list(values) -return self._with_new_scol(self.spark.column.isin([SF.lit(v) for v in values])) + +other = [SF.lit(v) for v in values] +scol = self.spark.column.isin(other) +field = self._internal.data_fields[0].copy( +dtype=np.dtype("bool"), spark_type=BooleanType(), nullable=False +) +return self._with_new_scol(scol=F.coalesce(scol, F.lit(False)), field=field) def isnull(self: IndexOpsLike) -> IndexOpsLike: """ diff --git a/python/pyspark/pandas/tests/test_series.py b/python/pyspark/pandas/tests/test_series.py index 1bf8388..09e5d30 100644 --- a/python/pyspark/pandas/tests/test_series.py +++ b/python/pyspark/pandas/tests/test_series.py @@ -394,6 +394,23 @@ class SeriesTest(PandasOnSparkTestCase, SQLTestUtils): with self.assertRaisesRegex(TypeError, msg): psser.isin(1) +# when Series have NaN +pser = pd.Series(["lama", "cow", None, "lama", "beetle", "lama", "hippo", None], name="a") +psser = ps.from_pandas(pser) + +self.assert_eq(psser.isin(["cow", "lama"]), pser.isin(["cow", "lama"])) + +pser = pd.Series([None, 5, None, 3, 2, 1, None, 0, 0], name="a") +psser = ps.from_pandas(pser) + +if LooseVersion(pd.__version__) >= LooseVersion("1.2"): +self.assert_eq(psser.isin([1, 5, 0, None]), pser.isin([1, 5, 0, None])) +else: +expected = pd.Series( +[False, True, False, False, False, True, False, True, True], name="a" +) +self.assert_eq(psser.isin([1, 5, 0, None]), expected) + def test_drop_duplicates(self): pdf = pd.DataFrame({"animal": ["lama", "cow", "lama", "beetle", "lama", "hippo"]}) psdf = ps.from_pandas(pdf) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch branch-3.2 updated: [SPARK-36722][PYTHON] Fix Series.update with another in same frame
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch branch-3.2 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.2 by this push: new 017bce7 [SPARK-36722][PYTHON] Fix Series.update with another in same frame 017bce7 is described below commit 017bce7b118cd643e652d5a7914294e281b05e6e Author: dgd-contributor AuthorDate: Wed Sep 15 11:08:01 2021 -0700 [SPARK-36722][PYTHON] Fix Series.update with another in same frame ### What changes were proposed in this pull request? Fix Series.update with another in same frame also add test for update series in diff frame ### Why are the changes needed? Fix Series.update with another in same frame Pandas behavior: ``` python >>> pdf = pd.DataFrame( ... {"a": [None, 2, 3, 4, 5, 6, 7, 8, None], "b": [None, 5, None, 3, 2, 1, None, 0, 0]}, ... ) >>> pdf ab 0 NaN NaN 1 2.0 5.0 2 3.0 NaN 3 4.0 3.0 4 5.0 2.0 5 6.0 1.0 6 7.0 NaN 7 8.0 0.0 8 NaN 0.0 >>> pdf.a.update(pdf.b) >>> pdf ab 0 NaN NaN 1 5.0 5.0 2 3.0 NaN 3 3.0 3.0 4 2.0 2.0 5 1.0 1.0 6 7.0 NaN 7 0.0 0.0 8 0.0 0.0 ``` ### Does this PR introduce _any_ user-facing change? Before ```python >>> psdf = ps.DataFrame( ... {"a": [None, 2, 3, 4, 5, 6, 7, 8, None], "b": [None, 5, None, 3, 2, 1, None, 0, 0]}, ... ) >>> psdf.a.update(psdf.b) Traceback (most recent call last): File "", line 1, in File "/Users/dgd/spark/python/pyspark/pandas/series.py", line 4551, in update combined = combine_frames(self._psdf, other._psdf, how="leftouter") File "/Users/dgd/spark/python/pyspark/pandas/utils.py", line 141, in combine_frames assert not same_anchor( AssertionError: We don't need to combine. `this` and `that` are same. >>> ``` After ```python >>> psdf = ps.DataFrame( ... {"a": [None, 2, 3, 4, 5, 6, 7, 8, None], "b": [None, 5, None, 3, 2, 1, None, 0, 0]}, ... ) >>> psdf.a.update(psdf.b) >>> psdf ab 0 NaN NaN 1 5.0 5.0 2 3.0 NaN 3 3.0 3.0 4 2.0 2.0 5 1.0 1.0 6 7.0 NaN 7 0.0 0.0 8 0.0 0.0 >>> ``` ### How was this patch tested? unit tests Closes #33968 from dgd-contributor/SPARK-36722_fix_update_same_anchor. Authored-by: dgd-contributor Signed-off-by: Takuya UESHIN (cherry picked from commit c15072cc7397cb59496b7da1153d663d8201865c) Signed-off-by: Takuya UESHIN --- python/pyspark/pandas/series.py| 35 ++ .../pandas/tests/test_ops_on_diff_frames.py| 9 ++ python/pyspark/pandas/tests/test_series.py | 32 3 files changed, 64 insertions(+), 12 deletions(-) diff --git a/python/pyspark/pandas/series.py b/python/pyspark/pandas/series.py index 568754c..0eebcc9 100644 --- a/python/pyspark/pandas/series.py +++ b/python/pyspark/pandas/series.py @@ -4498,22 +4498,33 @@ class Series(Frame, IndexOpsMixin, Generic[T]): if not isinstance(other, Series): raise TypeError("'other' must be a Series") -combined = combine_frames(self._psdf, other._psdf, how="leftouter") +if same_anchor(self, other): +scol = ( +F.when(other.spark.column.isNotNull(), other.spark.column) +.otherwise(self.spark.column) + .alias(self._psdf._internal.spark_column_name_for(self._column_label)) +) +internal = self._psdf._internal.with_new_spark_column( +self._column_label, scol # TODO: dtype? +) +self._psdf._update_internal_frame(internal) +else: +combined = combine_frames(self._psdf, other._psdf, how="leftouter") -this_scol = combined["this"]._internal.spark_column_for(self._column_label) -that_scol = combined["that"]._internal.spark_column_for(other._column_label) +this_scol = combined["this"]._internal.spark_column_for(self._column_label) +that_scol = combined["that"]._internal.spark_column_for(other._column_label) -scol = ( -F.when(that_scol.isNotNull(), that_scol) -.otherwise(this_scol) - .alias(self._psdf._internal.spark_column_name_for(self._column_label)) -) +scol = ( +F.when(that_scol.isNotNu
[spark] branch master updated: [SPARK-36722][PYTHON] Fix Series.update with another in same frame
This is an automated email from the ASF dual-hosted git repository. ueshin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new c15072c [SPARK-36722][PYTHON] Fix Series.update with another in same frame c15072c is described below commit c15072cc7397cb59496b7da1153d663d8201865c Author: dgd-contributor AuthorDate: Wed Sep 15 11:08:01 2021 -0700 [SPARK-36722][PYTHON] Fix Series.update with another in same frame ### What changes were proposed in this pull request? Fix Series.update with another in same frame also add test for update series in diff frame ### Why are the changes needed? Fix Series.update with another in same frame Pandas behavior: ``` python >>> pdf = pd.DataFrame( ... {"a": [None, 2, 3, 4, 5, 6, 7, 8, None], "b": [None, 5, None, 3, 2, 1, None, 0, 0]}, ... ) >>> pdf ab 0 NaN NaN 1 2.0 5.0 2 3.0 NaN 3 4.0 3.0 4 5.0 2.0 5 6.0 1.0 6 7.0 NaN 7 8.0 0.0 8 NaN 0.0 >>> pdf.a.update(pdf.b) >>> pdf ab 0 NaN NaN 1 5.0 5.0 2 3.0 NaN 3 3.0 3.0 4 2.0 2.0 5 1.0 1.0 6 7.0 NaN 7 0.0 0.0 8 0.0 0.0 ``` ### Does this PR introduce _any_ user-facing change? Before ```python >>> psdf = ps.DataFrame( ... {"a": [None, 2, 3, 4, 5, 6, 7, 8, None], "b": [None, 5, None, 3, 2, 1, None, 0, 0]}, ... ) >>> psdf.a.update(psdf.b) Traceback (most recent call last): File "", line 1, in File "/Users/dgd/spark/python/pyspark/pandas/series.py", line 4551, in update combined = combine_frames(self._psdf, other._psdf, how="leftouter") File "/Users/dgd/spark/python/pyspark/pandas/utils.py", line 141, in combine_frames assert not same_anchor( AssertionError: We don't need to combine. `this` and `that` are same. >>> ``` After ```python >>> psdf = ps.DataFrame( ... {"a": [None, 2, 3, 4, 5, 6, 7, 8, None], "b": [None, 5, None, 3, 2, 1, None, 0, 0]}, ... ) >>> psdf.a.update(psdf.b) >>> psdf ab 0 NaN NaN 1 5.0 5.0 2 3.0 NaN 3 3.0 3.0 4 2.0 2.0 5 1.0 1.0 6 7.0 NaN 7 0.0 0.0 8 0.0 0.0 >>> ``` ### How was this patch tested? unit tests Closes #33968 from dgd-contributor/SPARK-36722_fix_update_same_anchor. Authored-by: dgd-contributor Signed-off-by: Takuya UESHIN --- python/pyspark/pandas/series.py| 35 ++ .../pandas/tests/test_ops_on_diff_frames.py| 9 ++ python/pyspark/pandas/tests/test_series.py | 32 3 files changed, 64 insertions(+), 12 deletions(-) diff --git a/python/pyspark/pandas/series.py b/python/pyspark/pandas/series.py index 8cbfdbf..d72c08d 100644 --- a/python/pyspark/pandas/series.py +++ b/python/pyspark/pandas/series.py @@ -4536,22 +4536,33 @@ class Series(Frame, IndexOpsMixin, Generic[T]): if not isinstance(other, Series): raise TypeError("'other' must be a Series") -combined = combine_frames(self._psdf, other._psdf, how="leftouter") +if same_anchor(self, other): +scol = ( +F.when(other.spark.column.isNotNull(), other.spark.column) +.otherwise(self.spark.column) + .alias(self._psdf._internal.spark_column_name_for(self._column_label)) +) +internal = self._psdf._internal.with_new_spark_column( +self._column_label, scol # TODO: dtype? +) +self._psdf._update_internal_frame(internal) +else: +combined = combine_frames(self._psdf, other._psdf, how="leftouter") -this_scol = combined["this"]._internal.spark_column_for(self._column_label) -that_scol = combined["that"]._internal.spark_column_for(other._column_label) +this_scol = combined["this"]._internal.spark_column_for(self._column_label) +that_scol = combined["that"]._internal.spark_column_for(other._column_label) -scol = ( -F.when(that_scol.isNotNull(), that_scol) -.otherwise(this_scol) - .alias(self._psdf._internal.spark_column_name_for(self._column_label)) -) +scol = ( +F.when(that_scol.isNotNull(), that_scol) +.otherwise(this_scol) + .alias(self._psdf._internal.spark_column_name_for(self._column