Skip to content

Conversation

@zero323
Copy link
Member

@zero323 zero323 commented Jan 10, 2017

What changes were proposed in this pull request?

This PR adds udf decorator syntax as proposed in SPARK-19160.

This allows users to define UDF using simplified syntax:

from pyspark.sql.decorators import udf
    
@udf(IntegerType())
def add_one(x):
    """Adds one"""
    if x is not None:
        return x + 1

without need to define a separate function and udf.

How was this patch tested?

Existing unit tests to ensure backward compatibility and additional unit tests covering new functionality.

@SparkQA
Copy link

SparkQA commented Jan 10, 2017

Test build #71158 has finished for PR 16533 at commit 8645ed8.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think this should conflict with pyspark.sql.functions.udf. If this were defined, then it would break existing code that uses functions.udf. It may be possible to have a decorator definition that maintains that API and acts as a decorator, but otherwise this should be renamed to something like register_udf.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you also add docstrings for this?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Regarding name clashes I would like to keep it as a separated module and let user decide if prefer to import from sql.functions or sql.decorators though technically speaking, with "parenthless" version, we could just merge it with pyspark.sql.functions.udf. The problem is it becomes rather ugly to document because the argument can be either DataType or Callable.

This is why I wait with adding documentation but I pushed some doctests.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Of course it could be just moved to sql.functions and renamed explicitly as udf_decorator. Then user can use an alias to get shorter call:

from sql.functions import udf_decorator as udf

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think that the two should be merged. If they have the same name, then what works in one place breaks in others and the two are mutually exclusive. Our users share code quite a bit and it is common to copy & paste UDFs. That should be fine, or should result in "NameError: name 'udf' is not defined", rather than having different behavior.

With the changes to the decorator to support using it without parens, it's already close to being compatible with the current functions.udf.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

OK, I moved decorator to sql.functions as udf_decorator. Explicit is better than implicit, right? Nevertheless I am reluctant to further attempts to merge it with the main udf function.

Technically It is possible, but it will require all kinds of special cases. It would be manageable for now, when we only support returnType, but if we ever decide to add further arguments (like name) it would be a mess. Especially when project is committed to supporting legacy Python versions. Otherwise we could just use keyword only arguments:

def udf(func=None, *, returnType=StringType()):
    ... 

We could mimic that to some extent, without using backwards compatibility, by using keyword arguments:

def udf(func=None, **kwargs):
    returnType = kwargs.get("returnType", StringType())
    ...

but I don't think it is worth loosing explicit argument list.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you add a test case for the default return type? I think it currently requires empty-parens:

@udf()
def truncate(s):
    return s[:10]

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It does. In general decorators with arguments should be called with empty parens. This call is equivalent to:

udf()(truncate)

We can support parentheses-less call but it requires a bit ugly checks on runtime. I'll push an example implementation and additional tests in a moment.

@rdblue
Copy link
Contributor

rdblue commented Jan 11, 2017

I think if the udf method takes as its first arg func=None, then you can match the old API and take care of the empty-paren problem. If a callable isn't passed as the first arg, then return a decorator function. If a callable is passed in, then register it. You can do something similar with *args if you don't want to require passing returnType as a named arg.

@SparkQA
Copy link

SparkQA commented Jan 11, 2017

Test build #71201 has finished for PR 16533 at commit 8daa362.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Jan 11, 2017

Test build #71202 has finished for PR 16533 at commit d35cb06.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Jan 11, 2017

Test build #71227 has finished for PR 16533 at commit 8efc00b.

  • This patch fails Python style tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Jan 11, 2017

Test build #71228 has finished for PR 16533 at commit d96a63e.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@zero323 zero323 changed the title [SPARK-19160][PYTHON][SQL][WIP] Add udf decorator [SPARK-19160][PYTHON][SQL] Add udf decorator Jan 12, 2017
@zero323 zero323 changed the title [SPARK-19160][PYTHON][SQL] Add udf decorator [SPARK-19160][PYTHON][SQL][WIP] Add udf decorator Jan 12, 2017
@zero323
Copy link
Member Author

zero323 commented Jan 12, 2017

@rdblue Unified udf which supports all cases (udf(f), decorator with parentheses and without, decorator with returnType) could be implemented like this:

def _udf(f, returnType=StringType()):
    """Creates a :class:`Column` expression representing a user defined function (UDF).

    .. note:: The user-defined functions must be deterministic. Due to optimization,
        duplicate invocations may be eliminated or the function may even be invoked more times than
        it is present in the query.

    :param f: python function
    :param returnType: a :class:`pyspark.sql.types.DataType` object

    >>> from pyspark.sql.types import IntegerType
    >>> slen = udf(lambda s: len(s), IntegerType())
    >>> df.select(slen(df.name).alias('slen')).collect()
    [Row(slen=5), Row(slen=3)]
    >>> @udf
    ... def to_upper(s):
    ...     if s is not None:
    ...         return s.upper()
    ...
    >>> @udf(returnType=IntegerType())
    ... def add_one(x):
    ...     if x is not None:
    ...         return x + 1
    ...
    """
    return UserDefinedFunction(f, returnType)


@functools.wraps(_udf)
def udf(f=None, **kwargs):
    """A decorator version of pyspark.sql.functions.udf
    """
    returnType = kwargs.get("returnType", StringType())

    if f is None:
        return functools.partial(_udf, returnType=returnType)
    else:
        return _udf(f=f, returnType=returnType)

We can use functools.wraps to update docstring. There are two problems with this design:

  • Once again in legacy Python we won't get meaningful argument list.
  • It will break pre-existing code which consider returnType to be a positional argument (including a bunch of tests).

Do you think this is an acceptable trade-off?

@rdblue
Copy link
Contributor

rdblue commented Jan 19, 2017

@zero323, I think that the decorator and existing UDF factory method should be the same, but that we can't break existing code. Can you explain why this necessarily breaks code that relies on returnType as a positional arg? What about something like this:

@functools.wraps(_udf)
def udf(f=None, returnType=StringType()):
    """A decorator version of pyspark.sql.functions.udf
    """
    if f is None:
        return functools.partial(_udf, returnType=returnType)
    else:
        return _udf(f=f, returnType=returnType)

@zero323
Copy link
Member Author

zero323 commented Jan 19, 2017

@rdblue Good question. I vaguely remember I had some motivation to avoid this but now I cannot recall why. In general I really liked the initial approach because it allowed us to write:

@udf(IntegerType())
def identity(x):
    return x

instead of

@udf(returnType=IntegerType())
def identity(x):
    return x

There is another trick, which some libraries that depend heavily on decorators, use:

@functools.wraps(_udf)
def udf(f=None, returnType=StringType()):
    """A decorator version of pyspark.sql.functions.udf
    """
    if f is None or isinstance(f, DataType):
        return functools.partial(_udf, returnType=returnType)
    else:
        return _udf(f=f, returnType=returnType)

but it smells.

I think that:

@functools.wraps(_udf)
def udf(f=None, returnType=StringType()):
    """A decorator version of pyspark.sql.functions.udf
    """
    if  isinstance(f, DataType):
       raise TypeError("returnType with decorator should be provided as a keyword argument")
    if f is None:
        return functools.partial(_udf, returnType=returnType)
    else:
        return _udf(f=f, returnType=returnType)

could be an acceptable trade-off. It doesn't brake current API and clearly communicates possible issues.

@SparkQA
Copy link

SparkQA commented Jan 19, 2017

Test build #71678 has finished for PR 16533 at commit 88326fd.

  • This patch fails Python style tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@rdblue
Copy link
Contributor

rdblue commented Jan 19, 2017

Yeah, I added the isinstance(f, DataType) trick to my local tests. I'd add it if it were up to me, but I'm fine requiring returnType to be a keyword arg if you feel strongly about it.

Looks like tests are broken, maybe rebase?

@SparkQA
Copy link

SparkQA commented Jan 20, 2017

Test build #71682 has finished for PR 16533 at commit 4ed5f6f.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@zero323 zero323 changed the title [SPARK-19160][PYTHON][SQL][WIP] Add udf decorator [SPARK-19160][PYTHON][SQL] Add udf decorator Jan 20, 2017
@zero323
Copy link
Member Author

zero323 commented Jan 20, 2017

@rdblue You're probably right and I am overthinking this. After all the goal here is to make this as user friendly as possible, and while isinstance is not pretty it does the trick. If we ever decide to add more arguments these can be made keyword only to avoid ambiguity.

Rebased, squashed and tested. Thank you for your support and advice!

@SparkQA
Copy link

SparkQA commented Jan 20, 2017

Test build #71721 has finished for PR 16533 at commit f55c6ee.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@rdblue
Copy link
Contributor

rdblue commented Jan 20, 2017

+1

Looks good to me. Thanks, @zero323!

Copy link
Contributor

@holdenk holdenk left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

First quick skim, looks like a potentially useful feature - lets see where this goes. I'll do a more detailed pass after Spark Summit East :)

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Even though we've got this with an _ maybe through in the docstring its for internal use only? Also since this is for internal use only maybe most of this should be moved to the public udf function now instead of on the internal only method?

Unless we intend for users to call _udf in which case we need to give it a different name.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If we don't mind some discrepancy between docstring and usage (like I mentioned I believe that documenting internals with switching between decorator with partial and full application won't do us any good here) it can be pushed as an internal helper into udf.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I like this example as a doctest - but maybe good to also show these being applied in the doctest.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we keep the since annotation? Also maybe explain what these params are in the docstring (which can be based on the docstirng you currently have for _udf but updated).

Copy link
Member Author

@zero323 zero323 Jan 26, 2017

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We can, but I am not sure if we should keep the original version or acknowledge change of behavior and use potential merge version.

Regarding docstring I am a bit reluctant. It can be documented for dev purposes but any user facing doc will have to expose ugly implementation details.

def udf(
    f: typing.Union[None, typing.Callable, pyspark.sql.types.DataType],  
    returnType: pyspark.sql.types.DataType=StringType()
) -> Union[types.FunctionType, functools.partial]:
    ...

Maybe I a wrong but it can bring more confusion than it is worth. We can of course move _udf docstring and skip wrapping (we considered different implementations where signatures differed) but of course it won't be accurate.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe you can mention why this would be the case in the comment as well?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do you mean something like this?

@functools.wraps(_udf)
def udf(f=None, returnType=StringType()):
    """A decorator version of pyspark.sql.functions.udf
    """
    if (
            # Decorator as @udf() or @udf
            f is None
            # Decorator with positional DataType @udf(IntegerType())
            or isinstance(f, DataType)
    ):
    ...

@SparkQA
Copy link

SparkQA commented Jan 27, 2017

Test build #72072 has finished for PR 16533 at commit 2906889.

  • This patch fails PySpark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Jan 27, 2017

Test build #72073 has finished for PR 16533 at commit 218a800.

  • This patch fails PySpark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Jan 27, 2017

Test build #72074 has finished for PR 16533 at commit d8f7f44.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@rxin
Copy link
Contributor

rxin commented Jan 30, 2017

Can return type also take a string?

@zero323
Copy link
Member Author

zero323 commented Feb 3, 2017

@holdenk If believe this is a good as it gets at this moment.

@SparkQA
Copy link

SparkQA commented Feb 3, 2017

Test build #72331 has finished for PR 16533 at commit 7dc670d.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@holdenk
Copy link
Contributor

holdenk commented Feb 13, 2017

Ok back from the summit, let me take a look through - sorry for the slowness with my travel.

@holdenk
Copy link
Contributor

holdenk commented Feb 13, 2017

So one minor thing, I don't see support for taking the return type as string - was that decided to be dropped?

@zero323
Copy link
Member Author

zero323 commented Feb 13, 2017

@holdenk We cannot support it without modifying UserDefinedFunction first so I decided to open a separate PR for this: #16769

@holdenk
Copy link
Contributor

holdenk commented Feb 13, 2017

Great, I've merged that PR, can you update the instance checks here as required and add a test using the decorator with a string?

@SparkQA
Copy link

SparkQA commented Feb 13, 2017

Test build #72822 has finished for PR 16533 at commit cafdf44.

  • This patch fails Python style tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@zero323 zero323 force-pushed the SPARK-19160 branch 2 times, most recently from 4f5c643 to 01be3ec Compare February 13, 2017 19:18
@SparkQA
Copy link

SparkQA commented Feb 13, 2017

Test build #72821 has finished for PR 16533 at commit f9b1eb2.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@holdenk
Copy link
Contributor

holdenk commented Feb 13, 2017

Ok this looks pretty reasonable, but I'm going to give it a day incase anyone (e.g. @rdblue or @HyukjinKwon ) have any last comments or suggestions.

Thanks so much for working on this @zero323 its going to really improve the simplicity of writing UDFs for Python users :)

@SparkQA
Copy link

SparkQA commented Feb 13, 2017

Test build #72823 has finished for PR 16533 at commit 4f5c643.

  • This patch fails PySpark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Feb 13, 2017

Test build #72824 has finished for PR 16533 at commit 01be3ec.

  • This patch fails PySpark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Feb 13, 2017

Test build #72825 has finished for PR 16533 at commit 282e943.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@HyukjinKwon
Copy link
Member

Thanks for cc'ing me. I like this pythonic way. +1 and looks okay to me too despite of the trick in argument checking.

@holdenk
Copy link
Contributor

holdenk commented Feb 14, 2017

Great :) So the sql python tests have changed a bit which means this needs another update (sorry @zero323 ) give me a ping when its updated and I'll try and come back to this quickly :)

This PR adds `udf` decorator syntax as proposed in [SPARK-19160](https://issues.apache.org/jira/browse/SPARK-19160).

This allows users to define UDF using simplified syntax:

```
from pyspark.sql.decorators import udf

@udf(IntegerType())
def add_one(x):
    """Adds one"""
    if x is not None:
        return x + 1
```

without need to define a separate function and udf.

Tested wiht existing unit tests to ensure backward compatibility and additional unit tests covering new functionality.
@SparkQA
Copy link

SparkQA commented Feb 14, 2017

Test build #72885 has finished for PR 16533 at commit 8f8d39e.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@zero323
Copy link
Member Author

zero323 commented Feb 14, 2017

@holdenk done.

@holdenk
Copy link
Contributor

holdenk commented Feb 15, 2017

Thanks for working on this @zero323, merged to master :)

@asfgit asfgit closed this in c97f4e1 Feb 15, 2017
@zero323
Copy link
Member Author

zero323 commented Feb 15, 2017

Thanks @holdenk

cmonkey pushed a commit to cmonkey/spark that referenced this pull request Feb 16, 2017
## What changes were proposed in this pull request?

This PR adds `udf` decorator syntax as proposed in [SPARK-19160](https://issues.apache.org/jira/browse/SPARK-19160).

This allows users to define UDF using simplified syntax:

```python
from pyspark.sql.decorators import udf

udf(IntegerType())
def add_one(x):
    """Adds one"""
    if x is not None:
        return x + 1
 ```

without need to define a separate function and udf.

## How was this patch tested?

Existing unit tests to ensure backward compatibility and additional unit tests covering new functionality.

Author: zero323 <[email protected]>

Closes apache#16533 from zero323/SPARK-19160.
@zero323 zero323 deleted the SPARK-19160 branch April 6, 2017 10:59
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

7 participants