Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
8 changes: 5 additions & 3 deletions python/pyspark/sql/functions.py
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@
from pyspark import since, SparkContext
from pyspark.rdd import _prepare_for_python_RDD, ignore_unicode_prefix
from pyspark.serializers import PickleSerializer, AutoBatchedSerializer
from pyspark.sql.types import StringType
from pyspark.sql.types import StringType, DataType, _parse_datatype_string
from pyspark.sql.column import Column, _to_java_column, _to_seq
from pyspark.sql.dataframe import DataFrame

Expand Down Expand Up @@ -1825,7 +1825,9 @@ class UserDefinedFunction(object):
"""
def __init__(self, func, returnType, name=None):
self.func = func
self.returnType = returnType
self.returnType = (
returnType if isinstance(returnType, DataType)
else _parse_datatype_string(returnType))
# Stores UserDefinedPythonFunctions jobj, once initialized
self._judf_placeholder = None
self._name = name or (
Expand Down Expand Up @@ -1869,7 +1871,7 @@ def udf(f, returnType=StringType()):
it is present in the query.

:param f: python function
:param returnType: a :class:`pyspark.sql.types.DataType` object
:param returnType: a :class:`pyspark.sql.types.DataType` object or data type string.

>>> from pyspark.sql.types import IntegerType
>>> slen = udf(lambda s: len(s), IntegerType())
Expand Down
15 changes: 15 additions & 0 deletions python/pyspark/sql/tests.py
Original file line number Diff line number Diff line change
Expand Up @@ -489,6 +489,21 @@ def test_udf_defers_judf_initalization(self):
"judf should be initialized after UDF has been called."
)

def test_udf_with_string_return_type(self):
from pyspark.sql.functions import UserDefinedFunction

add_one = UserDefinedFunction(lambda x: x + 1, "integer")
make_pair = UserDefinedFunction(lambda x: (-x, x), "struct<x:integer,y:integer>")
make_array = UserDefinedFunction(
lambda x: [float(x) for x in range(x, x + 3)], "array<double>")

expected = (2, Row(x=-1, y=1), [1.0, 2.0, 3.0])
actual = (self.spark.range(1, 2).toDF("x")
.select(add_one("x"), make_pair("x"), make_array("x"))
.first())

self.assertTupleEqual(expected, actual)

def test_basic_functions(self):
rdd = self.sc.parallelize(['{"foo":"bar"}', '{"foo":"baz"}'])
df = self.spark.read.json(rdd)
Expand Down