From e0eeb0f89fffb52cd4d15970bdf00c3c5d1eea88 Mon Sep 17 00:00:00 2001 From: zero323 Date: Tue, 14 Feb 2017 09:46:22 -0800 Subject: [SPARK-19162][PYTHON][SQL] UserDefinedFunction should validate that func is callable ## What changes were proposed in this pull request? UDF constructor checks if `func` argument is callable and if it is not, fails fast instead of waiting for an action. ## How was this patch tested? Unit tests. Author: zero323 Closes #16535 from zero323/SPARK-19162. --- python/pyspark/sql/functions.py | 5 +++++ python/pyspark/sql/tests.py | 7 +++++++ 2 files changed, 12 insertions(+) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 5213a3c358..4f4ae10892 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1864,6 +1864,11 @@ class UserDefinedFunction(object): .. versionadded:: 1.3 """ def __init__(self, func, returnType, name=None): + if not callable(func): + raise TypeError( + "Not a function or callable (__call__ is not defined): " + "{0}".format(type(func))) + self.func = func self.returnType = ( returnType if isinstance(returnType, DataType) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index d9d03337ff..73721674f6 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -504,6 +504,13 @@ class SQLTests(ReusedPySparkTestCase): self.assertTupleEqual(expected, actual) + def test_udf_shouldnt_accept_noncallable_object(self): + from pyspark.sql.functions import UserDefinedFunction + from pyspark.sql.types import StringType + + non_callable = None + self.assertRaises(TypeError, UserDefinedFunction, non_callable, StringType()) + def test_basic_functions(self): rdd = self.sc.parallelize(['{"foo":"bar"}', '{"foo":"baz"}']) df = self.spark.read.json(rdd) -- cgit v1.2.3