-
Notifications
You must be signed in to change notification settings - Fork 28.4k
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-22939] [PySpark] Support Spark UDF in registerFunction #20137
Changes from all commits
8216b6b
0ecdf63
e8d0a4c
35e6a4a
b89b720
3208136
f099261
d1ba703
6ac25e6
85f11bf
78e9b2c
09a1b89
2482e6b
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -227,15 +227,15 @@ def dropGlobalTempView(self, viewName): | |
@ignore_unicode_prefix | ||
@since(2.0) | ||
def registerFunction(self, name, f, returnType=StringType()): | ||
"""Registers a python function (including lambda function) as a UDF | ||
so it can be used in SQL statements. | ||
"""Registers a Python function (including lambda function) or a :class:`UserDefinedFunction` | ||
as a UDF. The registered UDF can be used in SQL statement. | ||
|
||
In addition to a name and the function itself, the return type can be optionally specified. | ||
When the return type is not given it default to a string and conversion will automatically | ||
be done. For any other return type, the produced object must match the specified type. | ||
|
||
:param name: name of the UDF | ||
:param f: python function | ||
:param f: a Python function, or a wrapped/native UserDefinedFunction | ||
:param returnType: a :class:`pyspark.sql.types.DataType` object | ||
:return: a wrapped :class:`UserDefinedFunction` | ||
|
||
|
@@ -255,9 +255,26 @@ def registerFunction(self, name, f, returnType=StringType()): | |
>>> _ = spark.udf.register("stringLengthInt", len, IntegerType()) | ||
>>> spark.sql("SELECT stringLengthInt('test')").collect() | ||
[Row(stringLengthInt(test)=4)] | ||
|
||
>>> import random | ||
>>> from pyspark.sql.functions import udf | ||
>>> from pyspark.sql.types import IntegerType, StringType | ||
>>> random_udf = udf(lambda: random.randint(0, 100), IntegerType()).asNondeterministic() | ||
>>> newRandom_udf = spark.catalog.registerFunction("random_udf", random_udf, StringType()) | ||
>>> spark.sql("SELECT random_udf()").collect() # doctest: +SKIP | ||
[Row(random_udf()=u'82')] | ||
>>> spark.range(1).select(newRandom_udf()).collect() # doctest: +SKIP | ||
[Row(random_udf()=u'62')] | ||
""" | ||
udf = UserDefinedFunction(f, returnType=returnType, name=name, | ||
evalType=PythonEvalType.SQL_BATCHED_UDF) | ||
|
||
# This is to check whether the input function is a wrapped/native UserDefinedFunction | ||
if hasattr(f, 'asNondeterministic'): | ||
udf = UserDefinedFunction(f.func, returnType=returnType, name=name, | ||
evalType=PythonEvalType.SQL_BATCHED_UDF, | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. cc @ueshin @icexelloss , shall we support register pandas UDF here too? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. seems we can support it by just changing There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. +1 but I think there's no way to use a group map UDF in SQL syntax if I understood correctly. I think we can safely fail fast for now as well. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. SGTM There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Will support the pandas UDF as a separate PR. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. +1 too |
||
deterministic=f.deterministic) | ||
else: | ||
udf = UserDefinedFunction(f, returnType=returnType, name=name, | ||
evalType=PythonEvalType.SQL_BATCHED_UDF) | ||
self._jsparkSession.udf().registerPython(name, udf._judf) | ||
return udf._wrapped() | ||
|
||
|
Original file line number | Diff line number | Diff line change | ||
---|---|---|---|---|
|
@@ -56,7 +56,8 @@ def _create_udf(f, returnType, evalType): | |||
) | ||||
|
||||
# Set the name of the UserDefinedFunction object to be the name of function f | ||||
udf_obj = UserDefinedFunction(f, returnType=returnType, name=None, evalType=evalType) | ||||
udf_obj = UserDefinedFunction( | ||||
f, returnType=returnType, name=None, evalType=evalType, deterministic=True) | ||||
return udf_obj._wrapped() | ||||
|
||||
|
||||
|
@@ -67,8 +68,10 @@ class UserDefinedFunction(object): | |||
.. versionadded:: 1.3 | ||||
""" | ||||
def __init__(self, func, | ||||
returnType=StringType(), name=None, | ||||
evalType=PythonEvalType.SQL_BATCHED_UDF): | ||||
returnType=StringType(), | ||||
name=None, | ||||
evalType=PythonEvalType.SQL_BATCHED_UDF, | ||||
deterministic=True): | ||||
if not callable(func): | ||||
raise TypeError( | ||||
"Invalid function: not a function or callable (__call__ is not defined): " | ||||
|
@@ -92,7 +95,7 @@ def __init__(self, func, | |||
func.__name__ if hasattr(func, '__name__') | ||||
else func.__class__.__name__) | ||||
self.evalType = evalType | ||||
self._deterministic = True | ||||
self.deterministic = deterministic | ||||
|
||||
@property | ||||
def returnType(self): | ||||
|
@@ -130,14 +133,17 @@ def _create_judf(self): | |||
wrapped_func = _wrap_function(sc, self.func, self.returnType) | ||||
jdt = spark._jsparkSession.parseDataType(self.returnType.json()) | ||||
judf = sc._jvm.org.apache.spark.sql.execution.python.UserDefinedPythonFunction( | ||||
self._name, wrapped_func, jdt, self.evalType, self._deterministic) | ||||
self._name, wrapped_func, jdt, self.evalType, self.deterministic) | ||||
return judf | ||||
|
||||
def __call__(self, *cols): | ||||
judf = self._judf | ||||
sc = SparkContext._active_spark_context | ||||
return Column(judf.apply(_to_seq(sc, cols, _to_java_column))) | ||||
|
||||
# This function is for improving the online help system in the interactive interpreter. | ||||
# For example, the built-in help / pydoc.help. It wraps the UDF with the docstring and | ||||
# argument annotation. (See: SPARK-19161) | ||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think we can put this in the docstring of There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I do not want to expose these comments to the doc. |
||||
def _wrapped(self): | ||||
""" | ||||
Wrap this udf with a function and attach docstring from func | ||||
|
@@ -162,7 +168,8 @@ def wrapper(*args): | |||
wrapper.func = self.func | ||||
wrapper.returnType = self.returnType | ||||
wrapper.evalType = self.evalType | ||||
wrapper.asNondeterministic = self.asNondeterministic | ||||
wrapper.deterministic = self.deterministic | ||||
wrapper.asNondeterministic = lambda: self.asNondeterministic()._wrapped() | ||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Can we do: wrapper.asNondeterministic = functools.wraps(
self.asNondeterministic)(lambda: self.asNondeterministic()._wrapped()) So that it can produce a proper pydoc when we do There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. good to know the difference There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I will leave this unchanged. Maybe you can submit a follow-up PR to address it? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Definitely. Will give a try within the following week tho ... |
||||
|
||||
return wrapper | ||||
|
||||
|
@@ -172,5 +179,5 @@ def asNondeterministic(self): | |||
|
||||
.. versionadded:: 2.3 | ||||
""" | ||||
self._deterministic = False | ||||
self.deterministic = False | ||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Can we call it spark/sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonFunction.scala Line 33 in ff48b1b
The opposite works fine to me too if that's possible in any way. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
|
||||
return self |
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.
Let's fix the doc for this too. It says
:param f: python function
but we could describe that it takes Python native function, wrapped function andUserDefinedFunction
too.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.
ok