-
Notifications
You must be signed in to change notification settings - Fork 28.1k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-48834][SQL] Disable variant input/output to python scalar UDFs, UDTFs, UDAFs during query compilation #47253
Conversation
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
How about Python UDF/UDTFs?
from pyspark.sql import SparkSession | ||
from pyspark.sql.functions import udf | ||
|
||
# Intentionally uses SparkSession so one implementation can be shared with/without | ||
# Spark Connect. | ||
schema = ( | ||
SparkSession.active().range(0).select(udf(lambda x: x, returnType=ddl)("id")).schema | ||
) | ||
assert len(schema) == 1 | ||
return schema[0].dataType | ||
return _parse_datatype_string(ddl) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why is this change needed?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
If we want to do something like fromDDL("v variant")
, fromDDL
actually calls a udf
just to get the output schema. However, this PR disables variant output UDFs during planning, so we have to get the data type another way
I've discussed a bit with @HyukjinKwon offline last night, who suggested that we instead block variant ser/de, however I took a second look today and, unless I'm missing something, I believe this route is preferable:
- This will fail earlier during planning rather than execution
- Blocking during ser/de may require more code changes, because pandas udfs don't seem to have the same codepath as non-pandas udfs.
- the
_parse_datatype_string
also was made to work with spark connect (which was @HyukjinKwon's initial concern) in this pr
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can we make sure the behaivor of _parse_datatype_string
is the same as the original fromDDL
? My concern is that this might introduce unintentional behavior change for a public API.
What's the error message if we do fromDDL(a variant)
without this change?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
yeah makes sense. The function comment mentions that this was added for Spark 4.0.0 (which won't be GA until 2025 IIUC?) so I thought the risk of a breaking change for a public API would be a bit lower.
The error message without changing fromDDL
for something like "a int v variant" would be
org.apache.spark.sql.AnalysisException: [DATATYPE_MISMATCH.UNSUPPORTED_UDF_OUTPUT_TYPE] Cannot resolve "<lambda>(id)" due to data type mismatch: UDFs do not support 'STRUCT<a: INT, v: VARIANT>' as an output data type. SQLSTATE: 42K09
which is pretty confusing for fromDDL
in my opinion (i.e. why does fromDDL
need to call a UDF in the first place?)
It seems that the UDF returnType
implementation directly calls _parse_datatype_string
here.
I added an additional test to test that the old udf behavior matches the new behavior
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/PythonUDF.scala
Outdated
Show resolved
Hide resolved
cc @HyukjinKwon |
620c6ce
to
e43927c
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for adding the tests!
Merged to master. |
…, UDTFs, UDAFs during query compilation ### What changes were proposed in this pull request? Throws an exception if a variant is the input/output type to/from python UDF, UDAF, UDTF ### Why are the changes needed? currently, variant input/output types to scalar UDFs will fail during execution or return a `net.razorvine.pickle.objects.ClassDictConstructor` to the user python code. For a better UX, we should fail during query compilation for failures, and block returning `ClassDictConstructor` to user code as we one day want to actually return `VariantVal`s to the user code. ### Does this PR introduce _any_ user-facing change? yes - attempting to use variants in python UDFs will now throw an exception rather than returning a `ClassDictConstructor` as before. However, we want to make this change now as we one day want to be able to return `VariantVal`s to the user code and do not want users relying on this current behavior ### How was this patch tested? added UTs ### Was this patch authored or co-authored using generative AI tooling? no Closes apache#47253 from richardc-db/variant_scalar_udfs. Authored-by: Richard Chen <r.chen@databricks.com> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
What changes were proposed in this pull request?
Throws an exception if a variant is the input/output type to/from python UDF, UDAF, UDTF
Why are the changes needed?
currently, variant input/output types to scalar UDFs will fail during execution or return a
net.razorvine.pickle.objects.ClassDictConstructor
to the user python code. For a better UX, we should fail during query compilation for failures, and block returningClassDictConstructor
to user code as we one day want to actually returnVariantVal
s to the user code.Does this PR introduce any user-facing change?
yes - attempting to use variants in python UDFs will now throw an exception rather than returning a
ClassDictConstructor
as before. However, we want to make this change now as we one day want to be able to returnVariantVal
s to the user code and do not want users relying on this current behaviorHow was this patch tested?
added UTs
Was this patch authored or co-authored using generative AI tooling?
no