[ 
https://issues.apache.org/jira/browse/SPARK-23645?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16395390#comment-16395390
 ] 

Stu (Michael Stewart) edited comment on SPARK-23645 at 3/12/18 3:28 PM:
------------------------------------------------------------------------

[~hyukjin.kwon] thanks for the thoughts. it actually turned out to be easier 
than i'd expected to get most of the way there. the issue, as usual, is 
python2. i failed the existing unit tests on attempts to call 
`inspect.getargspec` on a callable class and on a partial function. in python 
these two concepts are oddly differentiated from functions. in python 3 it is 
handled seamlessly by `inspect.getfullargspec`. of course our friend getargspec 
is deprecated since 3.0 but there is really no alternative for py2. 

 

one middle ground that might be acceptable is to raise an error in python2 if a 
user passed keyword args to a partial fn object/callable object, but allow 
usage on functions. i suspect the vast majority of usecases of UDF in python 
rely on actual plain-old functions. this would be a clear functionality 
improvement over present for quite few loc.

 

that is:

py2 - raise error as mentioned above, otherwise handle functions with kwargs 
normally

py3 - everything just works

 

[https://github.com/apache/spark/pull/20798]


was (Author: mstewart141):
[~hyukjin.kwon] thanks for the thoughts. it actually turned out to be easier 
than i'd expected to get most of the way there. the issue, as usual, is 
python2. i failed the existing unit tests on attempts to call 
`inspect.getargspec` on a callable class and on a partial function. in python 
these two concepts are oddly differentiated from functions. in python 3 it is 
handled seamlessly by `inspect.getfullargspec`. of course our friend getargspec 
is deprecated since 3.0 but there is really no alternative for py2. 

 

one middle ground that might be acceptable is to raise an error in python2 if a 
user passed keyword args to a partial fn object/callable object, but allow 
usage on functions. i suspect the vast majority of usecases of UDF in python 
rely on actual plain-old functions. 

 

that is:

py2 - raise error as mentioned above, otherwise handle functions with kwargs 
normally

py3 - everything just works

 

https://github.com/apache/spark/pull/20798

> pandas_udf can not be called with keyword arguments
> ---------------------------------------------------
>
>                 Key: SPARK-23645
>                 URL: https://issues.apache.org/jira/browse/SPARK-23645
>             Project: Spark
>          Issue Type: Improvement
>          Components: PySpark
>    Affects Versions: 2.3.0
>         Environment: python 3.6 | pyspark 2.3.0 | Using Scala version 2.11.8, 
> OpenJDK 64-Bit Server VM, 1.8.0_141
>            Reporter: Stu (Michael Stewart)
>            Priority: Minor
>
> pandas_udf (all python udfs(?)) do not accept keyword arguments because 
> `pyspark/sql/udf.py` class `UserDefinedFunction` has __call__, and also 
> wrapper utility methods, that only accept args and not kwargs:
> @ line 168:
> {code:java}
> ...
> 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)
> def _wrapped(self):
>     """
>     Wrap this udf with a function and attach docstring from func
>     """
>     # It is possible for a callable instance without __name__ attribute or/and
>     # __module__ attribute to be wrapped here. For example, 
> functools.partial. In this case,
>     # we should avoid wrapping the attributes from the wrapped function to 
> the wrapper
>     # function. So, we take out these attribute names from the default names 
> to set and
>     # then manually assign it after being wrapped.
>     assignments = tuple(
>         a for a in functools.WRAPPER_ASSIGNMENTS if a != '__name__' and a != 
> '__module__')
>     @functools.wraps(self.func, assigned=assignments)
>     def wrapper(*args):
>         return self(*args)
> ...{code}
> as seen in:
> {code:java}
> from pyspark.sql import SparkSession
> from pyspark.sql.functions import pandas_udf, PandasUDFType, col, lit
> spark = SparkSession.builder.getOrCreate()
> df = spark.range(12).withColumn('b', col('id') * 2)
> def ok(a,b): return a*b
> df.withColumn('ok', pandas_udf(f=ok, returnType='bigint')('id','b')).show()  
> # no problems
> df.withColumn('ok', pandas_udf(f=ok, 
> returnType='bigint')(a='id',b='b')).show()  # fail with ~no stacktrace thanks 
> to wrapper helper
> ---------------------------------------------------------------------------
> TypeError Traceback (most recent call last)
> <ipython-input-2-8ba6c4344dc7> in <module>()
> ----> 1 df.withColumn('ok', pandas_udf(f=ok, 
> returnType='bigint')(a='id',b='b')).show()
> TypeError: wrapper() got an unexpected keyword argument 'a'{code}
>  
>  
> *discourse*: it isn't difficult to swap back in the kwargs, allowing the UDF 
> to be called as such, but the cols tuple that gets passed in the call method:
> {code:java}
> _to_seq(sc, cols, _to_java_column{code}
>  has to be in the right order based on the functions defined argument inputs, 
> or the function will return incorrect results. so, the challenge here is to:
> (a) make sure to reconstruct the proper order of the full args/kwargs
> --> args first, and then kwargs (not in the order passed but in the order 
> requested by the fn)
> (b) handle python2 and python3 `inspect` module inconsistencies 



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org

Reply via email to