ueshin commented on code in PR #48770:
URL: https://github.com/apache/spark/pull/48770#discussion_r1831904054
##########
python/pyspark/sql/tests/pandas/test_pandas_udf_scalar.py:
##########
@@ -752,46 +752,86 @@ def check_vectorized_udf_return_scalar(self):
def test_udf_with_variant_input(self):
df = self.spark.range(0, 10).selectExpr("parse_json(cast(id as
string)) v")
- from pyspark.sql.functions import col
- scalar_f = pandas_udf(lambda u: str(u), StringType())
+ scalar_f = pandas_udf(lambda u: u.apply(str), StringType(),
PandasUDFType.SCALAR)
iter_f = pandas_udf(
- lambda it: map(lambda u: str(u), it), StringType(),
PandasUDFType.SCALAR_ITER
+ lambda it: map(lambda u: u.apply(str), it), StringType(),
PandasUDFType.SCALAR_ITER
)
+ expected = [Row(udf="{0}".format(i)) for i in range(10)]
+
for f in [scalar_f, iter_f]:
- with self.assertRaises(AnalysisException) as ae:
- df.select(f(col("v"))).collect()
-
- self.check_error(
- exception=ae.exception,
- errorClass="DATATYPE_MISMATCH.UNSUPPORTED_UDF_INPUT_TYPE",
- messageParameters={
- "sqlExpr": '"<lambda>(v)"',
- "dataType": "VARIANT",
- },
- )
+ result = df.select(f(col("v")).alias("udf")).collect()
+ self.assertEqual(result, expected)
def test_udf_with_variant_output(self):
- # Corresponds to a JSON string of {"a": "b"}.
- returned_variant = VariantVal(bytes([2, 1, 0, 0, 2, 5, 98]), bytes([1,
1, 0, 1, 97]))
- scalar_f = pandas_udf(lambda x: returned_variant, VariantType())
+ scalar_f = pandas_udf(
+ lambda u: u.apply(lambda i: VariantVal(bytes([12, i]), bytes([1,
0, 0]))), VariantType()
+ )
iter_f = pandas_udf(
- lambda it: map(lambda x: returned_variant, it), VariantType(),
PandasUDFType.SCALAR_ITER
+ lambda it: map(lambda u: u.apply(
+ lambda i: VariantVal(bytes([12, i]), bytes([1, 0, 0]))
Review Comment:
What does `VariantVal(bytes([12, i]), bytes([1, 0, 0]))` mean?
How can we properly create `VariantVal` as a user?
##########
python/pyspark/sql/tests/test_udf.py:
##########
@@ -334,68 +334,118 @@ def test_udf_with_filter_function(self):
self.assertEqual(sel.collect(), [Row(key=1, value="1")])
def test_udf_with_variant_input(self):
- df = self.spark.range(0, 10).selectExpr("parse_json(cast(id as
string)) v")
-
- u = udf(lambda u: str(u), StringType())
- with self.assertRaises(AnalysisException) as ae:
- df.select(u(col("v"))).collect()
-
- self.check_error(
- exception=ae.exception,
- errorClass="DATATYPE_MISMATCH.UNSUPPORTED_UDF_INPUT_TYPE",
- messageParameters={"sqlExpr": '"<lambda>(v)"', "dataType":
"VARIANT"},
- )
+ for arrow_enabled in ["false", "true"]:
+ with self.sql_conf({"spark.sql.execution.pythonUDF.arrow.enabled":
arrow_enabled}):
Review Comment:
We don't need this here. There is `PythonUDFArrowTests` sharing tests in
`BaseUDFTestsMixin` with the config enabled.
##########
python/pyspark/sql/tests/test_udf.py:
##########
@@ -334,68 +334,118 @@ def test_udf_with_filter_function(self):
self.assertEqual(sel.collect(), [Row(key=1, value="1")])
def test_udf_with_variant_input(self):
- df = self.spark.range(0, 10).selectExpr("parse_json(cast(id as
string)) v")
-
- u = udf(lambda u: str(u), StringType())
- with self.assertRaises(AnalysisException) as ae:
- df.select(u(col("v"))).collect()
-
- self.check_error(
- exception=ae.exception,
- errorClass="DATATYPE_MISMATCH.UNSUPPORTED_UDF_INPUT_TYPE",
- messageParameters={"sqlExpr": '"<lambda>(v)"', "dataType":
"VARIANT"},
- )
+ for arrow_enabled in ["false", "true"]:
+ with self.sql_conf({"spark.sql.execution.pythonUDF.arrow.enabled":
arrow_enabled}):
+ df = self.spark.range(0, 10).selectExpr("parse_json(cast(id as
string)) v")
+ u = udf(lambda u: str(u), StringType())
+ expected = [Row(udf="{0}".format(i)) for i in range(10)]
+ result = df.select(u(col("v")).alias("udf")).collect()
+ self.assertEqual(result, expected)
def test_udf_with_complex_variant_input(self):
- df = self.spark.range(0, 10).selectExpr(
- "named_struct('v', parse_json(cast(id as string))) struct_of_v"
- )
-
- u = udf(lambda u: str(u), StringType())
-
- with self.assertRaises(AnalysisException) as ae:
- df.select(u(col("struct_of_v"))).collect()
-
- self.check_error(
- exception=ae.exception,
- errorClass="DATATYPE_MISMATCH.UNSUPPORTED_UDF_INPUT_TYPE",
- messageParameters={
- "sqlExpr": '"<lambda>(struct_of_v)"',
- "dataType": "STRUCT<v: VARIANT NOT NULL>",
- },
- )
+ for arrow_enabled in ["false", "true"]:
+ with self.sql_conf({"spark.sql.execution.pythonUDF.arrow.enabled":
arrow_enabled}):
Review Comment:
ditto, and following tests in `BaseUDFTestsMixin`.
--
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]