2929from pyspark .sql .readwriter import DataFrameReader
3030from pyspark .sql .streaming import DataStreamReader
3131from pyspark .sql .types import IntegerType , Row , StringType
32+ from pyspark .sql .udf import UDFRegistration
3233from pyspark .sql .utils import install_exception_handler
3334
34- __all__ = ["SQLContext" , "HiveContext" , "UDFRegistration" ]
35+ __all__ = ["SQLContext" , "HiveContext" ]
3536
3637
3738class SQLContext (object ):
@@ -147,7 +148,7 @@ def udf(self):
147148
148149 :return: :class:`UDFRegistration`
149150 """
150- return UDFRegistration ( self )
151+ return self . sparkSession . udf
151152
152153 @since (1.4 )
153154 def range (self , start , end = None , step = 1 , numPartitions = None ):
@@ -172,113 +173,29 @@ def range(self, start, end=None, step=1, numPartitions=None):
172173 """
173174 return self .sparkSession .range (start , end , step , numPartitions )
174175
175- @ignore_unicode_prefix
176176 @since (1.2 )
177177 def registerFunction (self , name , f , returnType = None ):
178- """Registers a Python function (including lambda function) or a :class:`UserDefinedFunction`
179- as a UDF. The registered UDF can be used in SQL statements.
180-
181- :func:`spark.udf.register` is an alias for :func:`sqlContext.registerFunction`.
182-
183- In addition to a name and the function itself, `returnType` can be optionally specified.
184- 1) When f is a Python function, `returnType` defaults to a string. The produced object must
185- match the specified type. 2) When f is a :class:`UserDefinedFunction`, Spark uses the return
186- type of the given UDF as the return type of the registered UDF. The input parameter
187- `returnType` is None by default. If given by users, the value must be None.
188-
189- :param name: name of the UDF in SQL statements.
190- :param f: a Python function, or a wrapped/native UserDefinedFunction. The UDF can be either
191- row-at-a-time or vectorized.
192- :param returnType: the return type of the registered UDF.
193- :return: a wrapped/native :class:`UserDefinedFunction`
194-
195- >>> strlen = sqlContext.registerFunction("stringLengthString", lambda x: len(x))
196- >>> sqlContext.sql("SELECT stringLengthString('test')").collect()
197- [Row(stringLengthString(test)=u'4')]
198-
199- >>> sqlContext.sql("SELECT 'foo' AS text").select(strlen("text")).collect()
200- [Row(stringLengthString(text)=u'3')]
201-
202- >>> from pyspark.sql.types import IntegerType
203- >>> _ = sqlContext.registerFunction("stringLengthInt", lambda x: len(x), IntegerType())
204- >>> sqlContext.sql("SELECT stringLengthInt('test')").collect()
205- [Row(stringLengthInt(test)=4)]
206-
207- >>> from pyspark.sql.types import IntegerType
208- >>> _ = sqlContext.udf.register("stringLengthInt", lambda x: len(x), IntegerType())
209- >>> sqlContext.sql("SELECT stringLengthInt('test')").collect()
210- [Row(stringLengthInt(test)=4)]
211-
212- >>> from pyspark.sql.types import IntegerType
213- >>> from pyspark.sql.functions import udf
214- >>> slen = udf(lambda s: len(s), IntegerType())
215- >>> _ = sqlContext.udf.register("slen", slen)
216- >>> sqlContext.sql("SELECT slen('test')").collect()
217- [Row(slen(test)=4)]
218-
219- >>> import random
220- >>> from pyspark.sql.functions import udf
221- >>> from pyspark.sql.types import IntegerType
222- >>> random_udf = udf(lambda: random.randint(0, 100), IntegerType()).asNondeterministic()
223- >>> new_random_udf = sqlContext.registerFunction("random_udf", random_udf)
224- >>> sqlContext.sql("SELECT random_udf()").collect() # doctest: +SKIP
225- [Row(random_udf()=82)]
226- >>> sqlContext.range(1).select(new_random_udf()).collect() # doctest: +SKIP
227- [Row(<lambda>()=26)]
228-
229- >>> from pyspark.sql.functions import pandas_udf, PandasUDFType
230- >>> @pandas_udf("integer", PandasUDFType.SCALAR) # doctest: +SKIP
231- ... def add_one(x):
232- ... return x + 1
233- ...
234- >>> _ = sqlContext.udf.register("add_one", add_one) # doctest: +SKIP
235- >>> sqlContext.sql("SELECT add_one(id) FROM range(3)").collect() # doctest: +SKIP
236- [Row(add_one(id)=1), Row(add_one(id)=2), Row(add_one(id)=3)]
178+ """An alias for :func:`spark.udf.register`.
179+ See :meth:`pyspark.sql.UDFRegistration.register`.
180+
181+ .. note:: Deprecated in 2.3.0. Use :func:`spark.udf.register` instead.
237182 """
238- return self .sparkSession .catalog .registerFunction (name , f , returnType )
183+ warnings .warn (
184+ "Deprecated in 2.3.0. Use spark.udf.register instead." ,
185+ DeprecationWarning )
186+ return self .sparkSession .udf .register (name , f , returnType )
239187
240- @ignore_unicode_prefix
241188 @since (2.1 )
242189 def registerJavaFunction (self , name , javaClassName , returnType = None ):
243- """Register a java UDF so it can be used in SQL statements.
244-
245- In addition to a name and the function itself, the return type can be optionally specified.
246- When the return type is not specified we would infer it via reflection.
247- :param name: name of the UDF
248- :param javaClassName: fully qualified name of java class
249- :param returnType: a :class:`pyspark.sql.types.DataType` object
250-
251- >>> sqlContext.registerJavaFunction("javaStringLength",
252- ... "test.org.apache.spark.sql.JavaStringLength", IntegerType())
253- >>> sqlContext.sql("SELECT javaStringLength('test')").collect()
254- [Row(UDF:javaStringLength(test)=4)]
255- >>> sqlContext.registerJavaFunction("javaStringLength2",
256- ... "test.org.apache.spark.sql.JavaStringLength")
257- >>> sqlContext.sql("SELECT javaStringLength2('test')").collect()
258- [Row(UDF:javaStringLength2(test)=4)]
190+ """An alias for :func:`spark.udf.registerJavaFunction`.
191+ See :meth:`pyspark.sql.UDFRegistration.registerJavaFunction`.
259192
193+ .. note:: Deprecated in 2.3.0. Use :func:`spark.udf.registerJavaFunction` instead.
260194 """
261- jdt = None
262- if returnType is not None :
263- jdt = self .sparkSession ._jsparkSession .parseDataType (returnType .json ())
264- self .sparkSession ._jsparkSession .udf ().registerJava (name , javaClassName , jdt )
265-
266- @ignore_unicode_prefix
267- @since (2.3 )
268- def registerJavaUDAF (self , name , javaClassName ):
269- """Register a java UDAF so it can be used in SQL statements.
270-
271- :param name: name of the UDAF
272- :param javaClassName: fully qualified name of java class
273-
274- >>> sqlContext.registerJavaUDAF("javaUDAF",
275- ... "test.org.apache.spark.sql.MyDoubleAvg")
276- >>> df = sqlContext.createDataFrame([(1, "a"),(2, "b"), (3, "a")],["id", "name"])
277- >>> df.registerTempTable("df")
278- >>> sqlContext.sql("SELECT name, javaUDAF(id) as avg from df group by name").collect()
279- [Row(name=u'b', avg=102.0), Row(name=u'a', avg=102.0)]
280- """
281- self .sparkSession ._jsparkSession .udf ().registerJavaUDAF (name , javaClassName )
195+ warnings .warn (
196+ "Deprecated in 2.3.0. Use spark.udf.registerJavaFunction instead." ,
197+ DeprecationWarning )
198+ return self .sparkSession .udf .registerJavaFunction (name , javaClassName , returnType )
282199
283200 # TODO(andrew): delete this once we refactor things to take in SparkSession
284201 def _inferSchema (self , rdd , samplingRatio = None ):
@@ -590,24 +507,6 @@ def refreshTable(self, tableName):
590507 self ._ssql_ctx .refreshTable (tableName )
591508
592509
593- class UDFRegistration (object ):
594- """Wrapper for user-defined function registration."""
595-
596- def __init__ (self , sqlContext ):
597- self .sqlContext = sqlContext
598-
599- def register (self , name , f , returnType = None ):
600- return self .sqlContext .registerFunction (name , f , returnType )
601-
602- def registerJavaFunction (self , name , javaClassName , returnType = None ):
603- self .sqlContext .registerJavaFunction (name , javaClassName , returnType )
604-
605- def registerJavaUDAF (self , name , javaClassName ):
606- self .sqlContext .registerJavaUDAF (name , javaClassName )
607-
608- register .__doc__ = SQLContext .registerFunction .__doc__
609-
610-
611510def _test ():
612511 import os
613512 import doctest
0 commit comments