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
3 changes: 2 additions & 1 deletion python/pyspark/sql/functions.py
Original file line number Diff line number Diff line change
Expand Up @@ -123,7 +123,8 @@ def _create_judf(self):
pickled_command, broadcast_vars, env, includes = _prepare_for_python_RDD(sc, command, self)
ssql_ctx = sc._jvm.SQLContext(sc._jsc.sc())
jdt = ssql_ctx.parseDataType(self.returnType.json())
judf = sc._jvm.UserDefinedPythonFunction(f.__name__, bytearray(pickled_command), env,
fname = f.__name__ if hasattr(f, '__name__') else f.__class__.__name__
judf = sc._jvm.UserDefinedPythonFunction(fname, bytearray(pickled_command), env,
includes, sc.pythonExec, broadcast_vars,
sc._javaAccumulator, jdt)
return judf
Expand Down
31 changes: 31 additions & 0 deletions python/pyspark/sql/tests.py
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import shutil
import tempfile
import pickle
import functools

import py4j

Expand All @@ -41,6 +42,7 @@
from pyspark.sql.types import *
from pyspark.sql.types import UserDefinedType, _infer_type
from pyspark.tests import ReusedPySparkTestCase
from pyspark.sql.functions import UserDefinedFunction


class ExamplePointUDT(UserDefinedType):
Expand Down Expand Up @@ -114,6 +116,35 @@ def tearDownClass(cls):
ReusedPySparkTestCase.tearDownClass()
shutil.rmtree(cls.tempdir.name, ignore_errors=True)

def test_udf_with_callable(self):
d = [Row(number=i, squared=i**2) for i in range(10)]
rdd = self.sc.parallelize(d)
data = self.sqlCtx.createDataFrame(rdd)

class PlusFour:
def __call__(self, col):
if col is not None:
return col + 4

call = PlusFour()
pudf = UserDefinedFunction(call, LongType())
res = data.select(pudf(data['number']).alias('plus_four'))
self.assertEqual(res.agg({'plus_four': 'sum'}).collect()[0][0], 85)

def test_udf_with_partial_function(self):
d = [Row(number=i, squared=i**2) for i in range(10)]
rdd = self.sc.parallelize(d)
data = self.sqlCtx.createDataFrame(rdd)

def some_func(col, param):
if col is not None:
return col + param

pfunc = functools.partial(some_func, param=4)
pudf = UserDefinedFunction(pfunc, LongType())
res = data.select(pudf(data['number']).alias('plus_four'))
self.assertEqual(res.agg({'plus_four': 'sum'}).collect()[0][0], 85)

def test_udf(self):
self.sqlCtx.registerFunction("twoArgs", lambda x, y: len(x) + y, IntegerType())
[row] = self.sqlCtx.sql("SELECT twoArgs('test', 1)").collect()
Expand Down