Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19325#discussion_r140624294
  
    --- Diff: python/pyspark/sql/functions.py ---
    @@ -2183,14 +2183,29 @@ def pandas_udf(f=None, returnType=StringType()):
         :param f: python function if used as a standalone function
         :param returnType: a :class:`pyspark.sql.types.DataType` object
     
    -    # TODO: doctest
    +    >>> from pyspark.sql.types import IntegerType, StringType
    +    >>> slen = pandas_udf(lambda s: s.str.len(), IntegerType())
    +    >>> @pandas_udf(returnType=StringType())
    +    ... def to_upper(s):
    +    ...     return s.str.upper()
    +    ...
    +    >>> @pandas_udf(returnType="integer")
    +    ... def add_one(x):
    +    ...     return x + 1
    +    ...
    +    >>> df = spark.createDataFrame([(1, "John Doe", 21)], ("id", "name", 
"age"))
    +    >>> df.select(slen("name").alias("slen(name)"), to_upper("name"), 
add_one("age")).show()
    +    +----------+--------------+------------+
    +    |slen(name)|to_upper(name)|add_one(age)|
    +    +----------+--------------+------------+
    +    |         8|      JOHN DOE|          22|
    +    +----------+--------------+------------+
         """
    +    wrapped_udf = _create_udf(f, returnType=returnType, vectorized=True)
         import inspect
    -    # If function "f" does not define the optional kwargs, then wrap with 
a kwargs placeholder
    -    if inspect.getargspec(f).keywords is None:
    -        return _create_udf(lambda *a, **kwargs: f(*a), 
returnType=returnType, vectorized=True)
    -    else:
    -        return _create_udf(f, returnType=returnType, vectorized=True)
    +    if not inspect.getargspec(wrapped_udf.func).args:
    --- End diff --
    
    It looks `wrapped_udf.func` could be `_udf` within `_create_udf`, that 
takes a single argument, for example:
    
    ```python
    @pandas_udf(returnType=LongType())
    def add_one():
        return 1
    ```
    
    I tried a rough idea to solve this:
    
    ```diff
    --- a/python/pyspark/sql/functions.py
    +++ b/python/pyspark/sql/functions.py
    @@ -2124,11 +2124,14 @@ class UserDefinedFunction(object):
             return wrapper
    
    
    -def _create_udf(f, returnType, vectorized):
    +def _create_udf(f, returnType, vectorized, checker=None):
    
         def _udf(f, returnType=StringType(), vectorized=vectorized):
             udf_obj = UserDefinedFunction(f, returnType, vectorized=vectorized)
    -        return udf_obj._wrapped()
    +        wrapped = udf_obj._wrapped()
    +        if checker is not None:
    +            checker(wrapped)
    +        return wrapped
    
         # decorator @udf, @udf(), @udf(dataType()), or similar with @pandas_udf
         if f is None or isinstance(f, (str, DataType)):
    @@ -2201,10 +2204,14 @@ def pandas_udf(f=None, returnType=StringType()):
         |         8|      JOHN DOE|          22|
         +----------+--------------+------------+
         """
    -    wrapped_udf = _create_udf(f, returnType=returnType, vectorized=True)
    -    import inspect
    -    if not inspect.getargspec(wrapped_udf.func).args:
    -        raise NotImplementedError("0-parameter pandas_udfs are not 
currently supported")
    +
    +    def checker(wrapped):
    +        import inspect
    +        if not inspect.getargspec(wrapped.func).args:
    +            raise NotImplementedError("0-parameter pandas_udfs are not 
currently supported")
    +
    +    wrapped_udf = _create_udf(f, returnType=returnType, vectorized=True, 
checker=checker)
    +
         return wrapped_udf
    ```


---

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

Reply via email to