zhengruifeng commented on code in PR #53666:
URL: https://github.com/apache/spark/pull/53666#discussion_r2660065753


##########
python/pyspark/worker.py:
##########
@@ -343,7 +365,7 @@ def evaluate(*args):
                 Takes list of Python objects and returns tuple of
                 (results, arrow_return_type, return_type).
                 """
-                return list(pool.map(lambda row: func(*row), get_args(*args)))
+                return list(pool.map(lambda row: coerce_result(func(*row)), 
get_args(*args)))

Review Comment:
   I feel such coercion should happen in `serializers.py`



##########
python/pyspark/sql/types.py:
##########
@@ -1052,6 +1256,28 @@ def fromInternal(self, obj: List[Optional[T]]) -> 
List[Optional[T]]:
             return obj
         return obj and [self.elementType.fromInternal(v) for v in obj]
 
+    def coerce(self, value: Any, policy: "CoercionPolicy" = 
CoercionPolicy.PERMISSIVE) -> Any:
+        if value is None:
+            return None
+        # list -> array: exact match
+        if isinstance(value, list):

Review Comment:
   should `elementType.coerce` be called here?



##########
python/pyspark/sql/types.py:
##########
@@ -1835,6 +2070,31 @@ def fromInternal(self, obj: Tuple) -> "Row":
             values = obj
         return _create_row(self.names, values)
 
+    def coerce(self, value: Any, policy: "CoercionPolicy" = 
CoercionPolicy.PERMISSIVE) -> Any:

Review Comment:
   do we need call `fields[...].dataType.coerce` to handle children fields?



##########
python/pyspark/worker.py:
##########
@@ -311,6 +315,7 @@ def wrap_arrow_batch_udf(f, args_offsets, kwargs_offsets, 
return_type, runner_co
 
 def wrap_arrow_batch_udf_arrow(f, args_offsets, kwargs_offsets, return_type, 
runner_conf):

Review Comment:
   this code path is for arrow-optimized python udf without legcay pandas 
conversion, there is another path `wrap_arrow_batch_udf_legacy` for python udf 
with legcay pandas conversion
   
   



##########
python/pyspark/sql/types.py:
##########
@@ -1203,6 +1429,15 @@ def fromInternal(self, obj: Dict[T, Optional[U]]) -> 
Dict[T, Optional[U]]:
             (self.keyType.fromInternal(k), self.valueType.fromInternal(v)) for 
k, v in obj.items()
         )
 
+    def coerce(self, value: Any, policy: "CoercionPolicy" = 
CoercionPolicy.PERMISSIVE) -> Any:

Review Comment:
   do we need to call `keyType.coerce` and `valueType.coerce`?



##########
python/pyspark/sql/tests/test_arrow_udf_coercion.py:
##########
@@ -0,0 +1,330 @@
+#
+# 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.
+#
+
+"""
+Integration tests for unified type coercion in Arrow-backed Python UDFs.
+
+These tests verify:
+1. PERMISSIVE policy: Arrow-enabled UDFs produce the same results as 
pickle-based UDFs
+2. WARN policy: Same as PERMISSIVE but with warnings (tested at unit level)
+3. STRICT policy: Arrow handles type conversion natively (no coercion applied)
+
+The goal is to ensure backward compatibility when enabling Arrow optimization.
+"""
+
+import array
+import datetime
+import re
+import unittest
+from decimal import Decimal
+
+from pyspark.sql import Row
+from pyspark.sql.functions import udf
+from pyspark.sql.types import (
+    ArrayType,
+    BinaryType,
+    BooleanType,
+    ByteType,
+    DateType,
+    DecimalType,
+    DoubleType,
+    FloatType,
+    IntegerType,
+    LongType,
+    MapType,
+    ShortType,
+    StringType,
+    StructField,
+    StructType,
+    TimestampType,
+)
+from pyspark.testing.utils import (
+    have_pyarrow,
+    have_pandas,
+    pyarrow_requirement_message,
+    pandas_requirement_message,
+)
+from pyspark.testing.sqlutils import ReusedSQLTestCase
+
+
+def normalize_result(value):
+    """Normalize result for comparison, handling Java object hash codes."""
+    result_str = repr(value)
+    # Normalize Java object hash codes to make tests deterministic
+    return re.sub(r"@[a-fA-F0-9]+", "@<hash>", result_str)
+
+
[email protected](
+    not have_pandas or not have_pyarrow,
+    pandas_requirement_message or pyarrow_requirement_message,
+)
+class ArrowUDFCoercionTests(ReusedSQLTestCase):

Review Comment:
   new test files should be listed in `modules.py`, otherwise it won't take 
effect in pyspark



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to