Repository: spark
Updated Branches:
  refs/heads/master 3c8650c12 -> 1881431dd


[SPARK-5274][SQL] Reconcile Java and Scala UDFRegistration.

As part of SPARK-5193:

1. Removed UDFRegistration as a mixin in SQLContext and made it a field ("udf").
2. For Java UDFs, renamed dataType to returnType.
3. For Scala UDFs, added type tags.
4. Added all Java UDF registration methods to Scala's UDFRegistration.
5. Documentation

Author: Reynold Xin <r...@databricks.com>

Closes #4056 from rxin/udf-registration and squashes the following commits:

ae9c556 [Reynold Xin] Updated example.
675a3c9 [Reynold Xin] Style fix
47c24ff [Reynold Xin] Python fix.
5f00c45 [Reynold Xin] Restore data type position in java udf and added typetags.
032f006 [Reynold Xin] [SPARK-5193][SQL] Reconcile Java and Scala 
UDFRegistration.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/1881431d
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/1881431d
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/1881431d

Branch: refs/heads/master
Commit: 1881431dd50e93a6948e4966d33742727f27e917
Parents: 3c8650c
Author: Reynold Xin <r...@databricks.com>
Authored: Thu Jan 15 16:15:12 2015 -0800
Committer: Reynold Xin <r...@databricks.com>
Committed: Thu Jan 15 16:15:12 2015 -0800

----------------------------------------------------------------------
 python/pyspark/sql.py                           |  16 +-
 .../scala/org/apache/spark/sql/SQLContext.scala |  29 +-
 .../org/apache/spark/sql/UdfRegistration.scala  | 692 +++++++++++++++++--
 .../org/apache/spark/sql/SQLQuerySuite.scala    |   2 +-
 .../scala/org/apache/spark/sql/UDFSuite.scala   |   9 +-
 .../apache/spark/sql/UserDefinedTypeSuite.scala |   2 +-
 .../spark/sql/hive/execution/HiveUdfSuite.scala |   2 +-
 7 files changed, 674 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/1881431d/python/pyspark/sql.py
----------------------------------------------------------------------
diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py
index 014ac17..dcd3b60 100644
--- a/python/pyspark/sql.py
+++ b/python/pyspark/sql.py
@@ -1281,14 +1281,14 @@ class SQLContext(object):
                                      self._sc._gateway._gateway_client)
         includes = ListConverter().convert(self._sc._python_includes,
                                            self._sc._gateway._gateway_client)
-        self._ssql_ctx.registerPython(name,
-                                      bytearray(pickled_command),
-                                      env,
-                                      includes,
-                                      self._sc.pythonExec,
-                                      broadcast_vars,
-                                      self._sc._javaAccumulator,
-                                      returnType.json())
+        self._ssql_ctx.udf().registerPython(name,
+                                            bytearray(pickled_command),
+                                            env,
+                                            includes,
+                                            self._sc.pythonExec,
+                                            broadcast_vars,
+                                            self._sc._javaAccumulator,
+                                            returnType.json())
 
     def inferSchema(self, rdd, samplingRatio=None):
         """Infer and apply a schema to an RDD of L{Row}.

http://git-wip-us.apache.org/repos/asf/spark/blob/1881431d/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
index 279671c..8ad1753 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
@@ -54,7 +54,6 @@ class SQLContext(@transient val sparkContext: SparkContext)
   extends org.apache.spark.Logging
   with CacheManager
   with ExpressionConversions
-  with UDFRegistration
   with Serializable {
 
   self =>
@@ -338,6 +337,34 @@ class SQLContext(@transient val sparkContext: SparkContext)
    */
   val experimental: ExperimentalMethods = new ExperimentalMethods(this)
 
+  /**
+   * A collection of methods for registering user-defined functions (UDF).
+   *
+   * The following example registers a Scala closure as UDF:
+   * {{{
+   *   sqlContext.udf.register("myUdf", (arg1: Int, arg2: String) => arg2 + 
arg1)
+   * }}}
+   *
+   * The following example registers a UDF in Java:
+   * {{{
+   *   sqlContext.udf().register("myUDF",
+   *     new UDF2<Integer, String, String>() {
+   *       @Override
+   *       public String call(Integer arg1, String arg2) {
+   *         return arg2 + arg1;
+   *       }
+   *     }, DataTypes.StringType);
+   * }}}
+   *
+   * Or, to use Java 8 lambda syntax:
+   * {{{
+   *   sqlContext.udf().register("myUDF",
+   *     (Integer arg1, String arg2) -> arg2 + arg1),
+   *     DataTypes.StringType);
+   * }}}
+   */
+  val udf: UDFRegistration = new UDFRegistration(this)
+
   protected[sql] class SparkPlanner extends SparkStrategies {
     val sparkContext: SparkContext = self.sparkContext
 

http://git-wip-us.apache.org/repos/asf/spark/blob/1881431d/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala
index 5fb4726..2e9d037 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala
@@ -19,22 +19,26 @@ package org.apache.spark.sql
 
 import java.util.{List => JList, Map => JMap}
 
+import scala.reflect.runtime.universe.TypeTag
+
 import org.apache.spark.Accumulator
 import org.apache.spark.api.python.PythonBroadcast
 import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.sql.api.java._
 import org.apache.spark.sql.catalyst.ScalaReflection
 import org.apache.spark.sql.catalyst.expressions.{Expression, ScalaUdf}
 import org.apache.spark.sql.execution.PythonUDF
+import org.apache.spark.sql.types.DataType
 
-import scala.reflect.runtime.universe.{TypeTag, typeTag}
 
 /**
- * Functions for registering scala lambda functions as UDFs in a SQLContext.
+ * Functions for registering user-defined functions.
  */
-private[sql] trait UDFRegistration {
-  self: SQLContext =>
+class UDFRegistration (sqlContext: SQLContext) extends 
org.apache.spark.Logging {
+
+  private val functionRegistry = sqlContext.functionRegistry
 
-  private[spark] def registerPython(
+  protected[sql] def registerPython(
       name: String,
       command: Array[Byte],
       envVars: JMap[String, String],
@@ -55,7 +59,7 @@ private[sql] trait UDFRegistration {
       """.stripMargin)
 
 
-    val dataType = parseDataType(stringDataType)
+    val dataType = sqlContext.parseDataType(stringDataType)
 
     def builder(e: Seq[Expression]) =
       PythonUDF(
@@ -72,133 +76,699 @@ private[sql] trait UDFRegistration {
     functionRegistry.registerFunction(name, builder)
   }
 
-  /** registerFunction 0-22 were generated by this script
+  // scalastyle:off
+
+  /* registerFunction 0-22 were generated by this script
 
     (0 to 22).map { x =>
-      val types = (1 to x).foldRight("T")((_, s) => {s"_, $s"})
-      s"""
-        def registerFunction[T: TypeTag](name: String, func: 
Function$x[$types]): Unit = {
-          def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[T].dataType, e)
+      val types = (1 to x).foldRight("RT")((i, s) => {s"A$i, $s"})
+      val typeTags = (1 to x).map(i => s"A${i}: TypeTag").foldLeft("RT: 
TypeTag")(_ + ", " + _)
+      val argDocs = (1 to x).map(i => s"         * @tparam A$i type of the UDF 
argument at position $i.").foldLeft("")(_ + "\n" + _)
+      println(s"""
+        /**
+         * Register a Scala closure of ${x} arguments as user-defined function 
(UDF).
+         * @tparam RT return type of UDF.$argDocs
+         */
+        def register[$typeTags](name: String, func: Function$x[$types]): Unit 
= {
+          def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[RT].dataType, e)
           functionRegistry.registerFunction(name, builder)
-        }
-      """
+        }""")
     }
-  */
 
-  // scalastyle:off
-  def registerFunction[T: TypeTag](name: String, func: Function0[T]): Unit = {
-    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[T].dataType, e)
+    (1 to 22).foreach { i =>
+      val extTypeArgs = (1 to i).map(_ => "_").mkString(", ")
+      val anyTypeArgs = (1 to i).map(_ => "Any").mkString(", ")
+      val anyCast = s".asInstanceOf[UDF$i[$anyTypeArgs, Any]]"
+      val anyParams = (1 to i).map(_ => "_: Any").mkString(", ")
+      println(s"""
+         |/**
+         | * Register a user-defined function with ${i} arguments.
+         | */
+         |def register(name: String, f: UDF$i[$extTypeArgs, _], returnType: 
DataType) = {
+         |  functionRegistry.registerFunction(
+         |    name,
+         |    (e: Seq[Expression]) => ScalaUdf(f$anyCast.call($anyParams), 
returnType, e))
+         |}""".stripMargin)
+    }
+    */
+
+  /**
+   * Register a Scala closure of 0 arguments as user-defined function (UDF).
+   * @tparam RT return type of UDF.
+   */
+  def register[RT: TypeTag](name: String, func: Function0[RT]): Unit = {
+    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[RT].dataType, e)
     functionRegistry.registerFunction(name, builder)
   }
 
-  def registerFunction[T: TypeTag](name: String, func: Function1[_, T]): Unit 
= {
-    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[T].dataType, e)
+  /**
+   * Register a Scala closure of 1 arguments as user-defined function (UDF).
+   * @tparam RT return type of UDF.
+   * @tparam A1 type of the UDF argument at position 1.
+   */
+  def register[RT: TypeTag, A1: TypeTag](name: String, func: Function1[A1, 
RT]): Unit = {
+    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[RT].dataType, e)
     functionRegistry.registerFunction(name, builder)
   }
 
-  def registerFunction[T: TypeTag](name: String, func: Function2[_, _, T]): 
Unit = {
-    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[T].dataType, e)
+  /**
+   * Register a Scala closure of 2 arguments as user-defined function (UDF).
+   * @tparam RT return type of UDF.
+   * @tparam A1 type of the UDF argument at position 1.
+   * @tparam A2 type of the UDF argument at position 2.
+   */
+  def register[RT: TypeTag, A1: TypeTag, A2: TypeTag](name: String, func: 
Function2[A1, A2, RT]): Unit = {
+    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[RT].dataType, e)
     functionRegistry.registerFunction(name, builder)
   }
 
-  def registerFunction[T: TypeTag](name: String, func: Function3[_, _, _, T]): 
Unit = {
-    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[T].dataType, e)
+  /**
+   * Register a Scala closure of 3 arguments as user-defined function (UDF).
+   * @tparam RT return type of UDF.
+   * @tparam A1 type of the UDF argument at position 1.
+   * @tparam A2 type of the UDF argument at position 2.
+   * @tparam A3 type of the UDF argument at position 3.
+   */
+  def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag](name: 
String, func: Function3[A1, A2, A3, RT]): Unit = {
+    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[RT].dataType, e)
     functionRegistry.registerFunction(name, builder)
   }
 
-  def registerFunction[T: TypeTag](name: String, func: Function4[_, _, _, _, 
T]): Unit = {
-    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[T].dataType, e)
+  /**
+   * Register a Scala closure of 4 arguments as user-defined function (UDF).
+   * @tparam RT return type of UDF.
+   * @tparam A1 type of the UDF argument at position 1.
+   * @tparam A2 type of the UDF argument at position 2.
+   * @tparam A3 type of the UDF argument at position 3.
+   * @tparam A4 type of the UDF argument at position 4.
+   */
+  def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: 
TypeTag](name: String, func: Function4[A1, A2, A3, A4, RT]): Unit = {
+    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[RT].dataType, e)
     functionRegistry.registerFunction(name, builder)
   }
 
-  def registerFunction[T: TypeTag](name: String, func: Function5[_, _, _, _, 
_, T]): Unit = {
-    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[T].dataType, e)
+  /**
+   * Register a Scala closure of 5 arguments as user-defined function (UDF).
+   * @tparam RT return type of UDF.
+   * @tparam A1 type of the UDF argument at position 1.
+   * @tparam A2 type of the UDF argument at position 2.
+   * @tparam A3 type of the UDF argument at position 3.
+   * @tparam A4 type of the UDF argument at position 4.
+   * @tparam A5 type of the UDF argument at position 5.
+   */
+  def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: 
TypeTag, A5: TypeTag](name: String, func: Function5[A1, A2, A3, A4, A5, RT]): 
Unit = {
+    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[RT].dataType, e)
     functionRegistry.registerFunction(name, builder)
   }
 
-  def registerFunction[T: TypeTag](name: String, func: Function6[_, _, _, _, 
_, _, T]): Unit = {
-    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[T].dataType, e)
+  /**
+   * Register a Scala closure of 6 arguments as user-defined function (UDF).
+   * @tparam RT return type of UDF.
+   * @tparam A1 type of the UDF argument at position 1.
+   * @tparam A2 type of the UDF argument at position 2.
+   * @tparam A3 type of the UDF argument at position 3.
+   * @tparam A4 type of the UDF argument at position 4.
+   * @tparam A5 type of the UDF argument at position 5.
+   * @tparam A6 type of the UDF argument at position 6.
+   */
+  def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: 
TypeTag, A5: TypeTag, A6: TypeTag](name: String, func: Function6[A1, A2, A3, 
A4, A5, A6, RT]): Unit = {
+    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[RT].dataType, e)
     functionRegistry.registerFunction(name, builder)
   }
 
-  def registerFunction[T: TypeTag](name: String, func: Function7[_, _, _, _, 
_, _, _, T]): Unit = {
-    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[T].dataType, e)
+  /**
+   * Register a Scala closure of 7 arguments as user-defined function (UDF).
+   * @tparam RT return type of UDF.
+   * @tparam A1 type of the UDF argument at position 1.
+   * @tparam A2 type of the UDF argument at position 2.
+   * @tparam A3 type of the UDF argument at position 3.
+   * @tparam A4 type of the UDF argument at position 4.
+   * @tparam A5 type of the UDF argument at position 5.
+   * @tparam A6 type of the UDF argument at position 6.
+   * @tparam A7 type of the UDF argument at position 7.
+   */
+  def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: 
TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag](name: String, func: 
Function7[A1, A2, A3, A4, A5, A6, A7, RT]): Unit = {
+    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[RT].dataType, e)
     functionRegistry.registerFunction(name, builder)
   }
 
-  def registerFunction[T: TypeTag](name: String, func: Function8[_, _, _, _, 
_, _, _, _, T]): Unit = {
-    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[T].dataType, e)
+  /**
+   * Register a Scala closure of 8 arguments as user-defined function (UDF).
+   * @tparam RT return type of UDF.
+   * @tparam A1 type of the UDF argument at position 1.
+   * @tparam A2 type of the UDF argument at position 2.
+   * @tparam A3 type of the UDF argument at position 3.
+   * @tparam A4 type of the UDF argument at position 4.
+   * @tparam A5 type of the UDF argument at position 5.
+   * @tparam A6 type of the UDF argument at position 6.
+   * @tparam A7 type of the UDF argument at position 7.
+   * @tparam A8 type of the UDF argument at position 8.
+   */
+  def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: 
TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag](name: String, 
func: Function8[A1, A2, A3, A4, A5, A6, A7, A8, RT]): Unit = {
+    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[RT].dataType, e)
     functionRegistry.registerFunction(name, builder)
   }
 
-  def registerFunction[T: TypeTag](name: String, func: Function9[_, _, _, _, 
_, _, _, _, _, T]): Unit = {
-    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[T].dataType, e)
+  /**
+   * Register a Scala closure of 9 arguments as user-defined function (UDF).
+   * @tparam RT return type of UDF.
+   * @tparam A1 type of the UDF argument at position 1.
+   * @tparam A2 type of the UDF argument at position 2.
+   * @tparam A3 type of the UDF argument at position 3.
+   * @tparam A4 type of the UDF argument at position 4.
+   * @tparam A5 type of the UDF argument at position 5.
+   * @tparam A6 type of the UDF argument at position 6.
+   * @tparam A7 type of the UDF argument at position 7.
+   * @tparam A8 type of the UDF argument at position 8.
+   * @tparam A9 type of the UDF argument at position 9.
+   */
+  def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: 
TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag](name: 
String, func: Function9[A1, A2, A3, A4, A5, A6, A7, A8, A9, RT]): Unit = {
+    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[RT].dataType, e)
     functionRegistry.registerFunction(name, builder)
   }
 
-  def registerFunction[T: TypeTag](name: String, func: Function10[_, _, _, _, 
_, _, _, _, _, _, T]): Unit = {
-    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[T].dataType, e)
+  /**
+   * Register a Scala closure of 10 arguments as user-defined function (UDF).
+   * @tparam RT return type of UDF.
+   * @tparam A1 type of the UDF argument at position 1.
+   * @tparam A2 type of the UDF argument at position 2.
+   * @tparam A3 type of the UDF argument at position 3.
+   * @tparam A4 type of the UDF argument at position 4.
+   * @tparam A5 type of the UDF argument at position 5.
+   * @tparam A6 type of the UDF argument at position 6.
+   * @tparam A7 type of the UDF argument at position 7.
+   * @tparam A8 type of the UDF argument at position 8.
+   * @tparam A9 type of the UDF argument at position 9.
+   * @tparam A10 type of the UDF argument at position 10.
+   */
+  def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: 
TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: 
TypeTag](name: String, func: Function10[A1, A2, A3, A4, A5, A6, A7, A8, A9, 
A10, RT]): Unit = {
+    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[RT].dataType, e)
     functionRegistry.registerFunction(name, builder)
   }
 
-  def registerFunction[T: TypeTag](name: String, func: Function11[_, _, _, _, 
_, _, _, _, _, _, _, T]): Unit = {
-    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[T].dataType, e)
+  /**
+   * Register a Scala closure of 11 arguments as user-defined function (UDF).
+   * @tparam RT return type of UDF.
+   * @tparam A1 type of the UDF argument at position 1.
+   * @tparam A2 type of the UDF argument at position 2.
+   * @tparam A3 type of the UDF argument at position 3.
+   * @tparam A4 type of the UDF argument at position 4.
+   * @tparam A5 type of the UDF argument at position 5.
+   * @tparam A6 type of the UDF argument at position 6.
+   * @tparam A7 type of the UDF argument at position 7.
+   * @tparam A8 type of the UDF argument at position 8.
+   * @tparam A9 type of the UDF argument at position 9.
+   * @tparam A10 type of the UDF argument at position 10.
+   * @tparam A11 type of the UDF argument at position 11.
+   */
+  def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: 
TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: 
TypeTag, A11: TypeTag](name: String, func: Function11[A1, A2, A3, A4, A5, A6, 
A7, A8, A9, A10, A11, RT]): Unit = {
+    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[RT].dataType, e)
     functionRegistry.registerFunction(name, builder)
   }
 
-  def registerFunction[T: TypeTag](name: String, func: Function12[_, _, _, _, 
_, _, _, _, _, _, _, _, T]): Unit = {
-    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[T].dataType, e)
+  /**
+   * Register a Scala closure of 12 arguments as user-defined function (UDF).
+   * @tparam RT return type of UDF.
+   * @tparam A1 type of the UDF argument at position 1.
+   * @tparam A2 type of the UDF argument at position 2.
+   * @tparam A3 type of the UDF argument at position 3.
+   * @tparam A4 type of the UDF argument at position 4.
+   * @tparam A5 type of the UDF argument at position 5.
+   * @tparam A6 type of the UDF argument at position 6.
+   * @tparam A7 type of the UDF argument at position 7.
+   * @tparam A8 type of the UDF argument at position 8.
+   * @tparam A9 type of the UDF argument at position 9.
+   * @tparam A10 type of the UDF argument at position 10.
+   * @tparam A11 type of the UDF argument at position 11.
+   * @tparam A12 type of the UDF argument at position 12.
+   */
+  def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: 
TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: 
TypeTag, A11: TypeTag, A12: TypeTag](name: String, func: Function12[A1, A2, A3, 
A4, A5, A6, A7, A8, A9, A10, A11, A12, RT]): Unit = {
+    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[RT].dataType, e)
     functionRegistry.registerFunction(name, builder)
   }
 
-  def registerFunction[T: TypeTag](name: String, func: Function13[_, _, _, _, 
_, _, _, _, _, _, _, _, _, T]): Unit = {
-    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[T].dataType, e)
+  /**
+   * Register a Scala closure of 13 arguments as user-defined function (UDF).
+   * @tparam RT return type of UDF.
+   * @tparam A1 type of the UDF argument at position 1.
+   * @tparam A2 type of the UDF argument at position 2.
+   * @tparam A3 type of the UDF argument at position 3.
+   * @tparam A4 type of the UDF argument at position 4.
+   * @tparam A5 type of the UDF argument at position 5.
+   * @tparam A6 type of the UDF argument at position 6.
+   * @tparam A7 type of the UDF argument at position 7.
+   * @tparam A8 type of the UDF argument at position 8.
+   * @tparam A9 type of the UDF argument at position 9.
+   * @tparam A10 type of the UDF argument at position 10.
+   * @tparam A11 type of the UDF argument at position 11.
+   * @tparam A12 type of the UDF argument at position 12.
+   * @tparam A13 type of the UDF argument at position 13.
+   */
+  def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: 
TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: 
TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag](name: String, func: 
Function13[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, RT]): Unit = 
{
+    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[RT].dataType, e)
     functionRegistry.registerFunction(name, builder)
   }
 
-  def registerFunction[T: TypeTag](name: String, func: Function14[_, _, _, _, 
_, _, _, _, _, _, _, _, _, _, T]): Unit = {
-    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[T].dataType, e)
+  /**
+   * Register a Scala closure of 14 arguments as user-defined function (UDF).
+   * @tparam RT return type of UDF.
+   * @tparam A1 type of the UDF argument at position 1.
+   * @tparam A2 type of the UDF argument at position 2.
+   * @tparam A3 type of the UDF argument at position 3.
+   * @tparam A4 type of the UDF argument at position 4.
+   * @tparam A5 type of the UDF argument at position 5.
+   * @tparam A6 type of the UDF argument at position 6.
+   * @tparam A7 type of the UDF argument at position 7.
+   * @tparam A8 type of the UDF argument at position 8.
+   * @tparam A9 type of the UDF argument at position 9.
+   * @tparam A10 type of the UDF argument at position 10.
+   * @tparam A11 type of the UDF argument at position 11.
+   * @tparam A12 type of the UDF argument at position 12.
+   * @tparam A13 type of the UDF argument at position 13.
+   * @tparam A14 type of the UDF argument at position 14.
+   */
+  def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: 
TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: 
TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag](name: String, 
func: Function14[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, 
RT]): Unit = {
+    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[RT].dataType, e)
     functionRegistry.registerFunction(name, builder)
   }
 
-  def registerFunction[T: TypeTag](name: String, func: Function15[_, _, _, _, 
_, _, _, _, _, _, _, _, _, _, _, T]): Unit = {
-    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[T].dataType, e)
+  /**
+   * Register a Scala closure of 15 arguments as user-defined function (UDF).
+   * @tparam RT return type of UDF.
+   * @tparam A1 type of the UDF argument at position 1.
+   * @tparam A2 type of the UDF argument at position 2.
+   * @tparam A3 type of the UDF argument at position 3.
+   * @tparam A4 type of the UDF argument at position 4.
+   * @tparam A5 type of the UDF argument at position 5.
+   * @tparam A6 type of the UDF argument at position 6.
+   * @tparam A7 type of the UDF argument at position 7.
+   * @tparam A8 type of the UDF argument at position 8.
+   * @tparam A9 type of the UDF argument at position 9.
+   * @tparam A10 type of the UDF argument at position 10.
+   * @tparam A11 type of the UDF argument at position 11.
+   * @tparam A12 type of the UDF argument at position 12.
+   * @tparam A13 type of the UDF argument at position 13.
+   * @tparam A14 type of the UDF argument at position 14.
+   * @tparam A15 type of the UDF argument at position 15.
+   */
+  def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: 
TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: 
TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: 
TypeTag](name: String, func: Function15[A1, A2, A3, A4, A5, A6, A7, A8, A9, 
A10, A11, A12, A13, A14, A15, RT]): Unit = {
+    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[RT].dataType, e)
     functionRegistry.registerFunction(name, builder)
   }
 
-  def registerFunction[T: TypeTag](name: String, func: Function16[_, _, _, _, 
_, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = {
-    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[T].dataType, e)
+  /**
+   * Register a Scala closure of 16 arguments as user-defined function (UDF).
+   * @tparam RT return type of UDF.
+   * @tparam A1 type of the UDF argument at position 1.
+   * @tparam A2 type of the UDF argument at position 2.
+   * @tparam A3 type of the UDF argument at position 3.
+   * @tparam A4 type of the UDF argument at position 4.
+   * @tparam A5 type of the UDF argument at position 5.
+   * @tparam A6 type of the UDF argument at position 6.
+   * @tparam A7 type of the UDF argument at position 7.
+   * @tparam A8 type of the UDF argument at position 8.
+   * @tparam A9 type of the UDF argument at position 9.
+   * @tparam A10 type of the UDF argument at position 10.
+   * @tparam A11 type of the UDF argument at position 11.
+   * @tparam A12 type of the UDF argument at position 12.
+   * @tparam A13 type of the UDF argument at position 13.
+   * @tparam A14 type of the UDF argument at position 14.
+   * @tparam A15 type of the UDF argument at position 15.
+   * @tparam A16 type of the UDF argument at position 16.
+   */
+  def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: 
TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: 
TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, 
A16: TypeTag](name: String, func: Function16[A1, A2, A3, A4, A5, A6, A7, A8, 
A9, A10, A11, A12, A13, A14, A15, A16, RT]): Unit = {
+    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[RT].dataType, e)
     functionRegistry.registerFunction(name, builder)
   }
 
-  def registerFunction[T: TypeTag](name: String, func: Function17[_, _, _, _, 
_, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = {
-    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[T].dataType, e)
+  /**
+   * Register a Scala closure of 17 arguments as user-defined function (UDF).
+   * @tparam RT return type of UDF.
+   * @tparam A1 type of the UDF argument at position 1.
+   * @tparam A2 type of the UDF argument at position 2.
+   * @tparam A3 type of the UDF argument at position 3.
+   * @tparam A4 type of the UDF argument at position 4.
+   * @tparam A5 type of the UDF argument at position 5.
+   * @tparam A6 type of the UDF argument at position 6.
+   * @tparam A7 type of the UDF argument at position 7.
+   * @tparam A8 type of the UDF argument at position 8.
+   * @tparam A9 type of the UDF argument at position 9.
+   * @tparam A10 type of the UDF argument at position 10.
+   * @tparam A11 type of the UDF argument at position 11.
+   * @tparam A12 type of the UDF argument at position 12.
+   * @tparam A13 type of the UDF argument at position 13.
+   * @tparam A14 type of the UDF argument at position 14.
+   * @tparam A15 type of the UDF argument at position 15.
+   * @tparam A16 type of the UDF argument at position 16.
+   * @tparam A17 type of the UDF argument at position 17.
+   */
+  def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: 
TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: 
TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, 
A16: TypeTag, A17: TypeTag](name: String, func: Function17[A1, A2, A3, A4, A5, 
A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, RT]): Unit = {
+    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[RT].dataType, e)
     functionRegistry.registerFunction(name, builder)
   }
 
-  def registerFunction[T: TypeTag](name: String, func: Function18[_, _, _, _, 
_, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = {
-    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[T].dataType, e)
+  /**
+   * Register a Scala closure of 18 arguments as user-defined function (UDF).
+   * @tparam RT return type of UDF.
+   * @tparam A1 type of the UDF argument at position 1.
+   * @tparam A2 type of the UDF argument at position 2.
+   * @tparam A3 type of the UDF argument at position 3.
+   * @tparam A4 type of the UDF argument at position 4.
+   * @tparam A5 type of the UDF argument at position 5.
+   * @tparam A6 type of the UDF argument at position 6.
+   * @tparam A7 type of the UDF argument at position 7.
+   * @tparam A8 type of the UDF argument at position 8.
+   * @tparam A9 type of the UDF argument at position 9.
+   * @tparam A10 type of the UDF argument at position 10.
+   * @tparam A11 type of the UDF argument at position 11.
+   * @tparam A12 type of the UDF argument at position 12.
+   * @tparam A13 type of the UDF argument at position 13.
+   * @tparam A14 type of the UDF argument at position 14.
+   * @tparam A15 type of the UDF argument at position 15.
+   * @tparam A16 type of the UDF argument at position 16.
+   * @tparam A17 type of the UDF argument at position 17.
+   * @tparam A18 type of the UDF argument at position 18.
+   */
+  def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: 
TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: 
TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, 
A16: TypeTag, A17: TypeTag, A18: TypeTag](name: String, func: Function18[A1, 
A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, 
RT]): Unit = {
+    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[RT].dataType, e)
     functionRegistry.registerFunction(name, builder)
   }
 
-  def registerFunction[T: TypeTag](name: String, func: Function19[_, _, _, _, 
_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = {
-    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[T].dataType, e)
+  /**
+   * Register a Scala closure of 19 arguments as user-defined function (UDF).
+   * @tparam RT return type of UDF.
+   * @tparam A1 type of the UDF argument at position 1.
+   * @tparam A2 type of the UDF argument at position 2.
+   * @tparam A3 type of the UDF argument at position 3.
+   * @tparam A4 type of the UDF argument at position 4.
+   * @tparam A5 type of the UDF argument at position 5.
+   * @tparam A6 type of the UDF argument at position 6.
+   * @tparam A7 type of the UDF argument at position 7.
+   * @tparam A8 type of the UDF argument at position 8.
+   * @tparam A9 type of the UDF argument at position 9.
+   * @tparam A10 type of the UDF argument at position 10.
+   * @tparam A11 type of the UDF argument at position 11.
+   * @tparam A12 type of the UDF argument at position 12.
+   * @tparam A13 type of the UDF argument at position 13.
+   * @tparam A14 type of the UDF argument at position 14.
+   * @tparam A15 type of the UDF argument at position 15.
+   * @tparam A16 type of the UDF argument at position 16.
+   * @tparam A17 type of the UDF argument at position 17.
+   * @tparam A18 type of the UDF argument at position 18.
+   * @tparam A19 type of the UDF argument at position 19.
+   */
+  def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: 
TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: 
TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, 
A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag](name: String, func: 
Function19[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, 
A16, A17, A18, A19, RT]): Unit = {
+    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[RT].dataType, e)
     functionRegistry.registerFunction(name, builder)
   }
 
-  def registerFunction[T: TypeTag](name: String, func: Function20[_, _, _, _, 
_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = {
-    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[T].dataType, e)
+  /**
+   * Register a Scala closure of 20 arguments as user-defined function (UDF).
+   * @tparam RT return type of UDF.
+   * @tparam A1 type of the UDF argument at position 1.
+   * @tparam A2 type of the UDF argument at position 2.
+   * @tparam A3 type of the UDF argument at position 3.
+   * @tparam A4 type of the UDF argument at position 4.
+   * @tparam A5 type of the UDF argument at position 5.
+   * @tparam A6 type of the UDF argument at position 6.
+   * @tparam A7 type of the UDF argument at position 7.
+   * @tparam A8 type of the UDF argument at position 8.
+   * @tparam A9 type of the UDF argument at position 9.
+   * @tparam A10 type of the UDF argument at position 10.
+   * @tparam A11 type of the UDF argument at position 11.
+   * @tparam A12 type of the UDF argument at position 12.
+   * @tparam A13 type of the UDF argument at position 13.
+   * @tparam A14 type of the UDF argument at position 14.
+   * @tparam A15 type of the UDF argument at position 15.
+   * @tparam A16 type of the UDF argument at position 16.
+   * @tparam A17 type of the UDF argument at position 17.
+   * @tparam A18 type of the UDF argument at position 18.
+   * @tparam A19 type of the UDF argument at position 19.
+   * @tparam A20 type of the UDF argument at position 20.
+   */
+  def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: 
TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: 
TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, 
A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag](name: 
String, func: Function20[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, 
A13, A14, A15, A16, A17, A18, A19, A20, RT]): Unit = {
+    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[RT].dataType, e)
     functionRegistry.registerFunction(name, builder)
   }
 
-  def registerFunction[T: TypeTag](name: String, func: Function21[_, _, _, _, 
_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = {
-    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[T].dataType, e)
+  /**
+   * Register a Scala closure of 21 arguments as user-defined function (UDF).
+   * @tparam RT return type of UDF.
+   * @tparam A1 type of the UDF argument at position 1.
+   * @tparam A2 type of the UDF argument at position 2.
+   * @tparam A3 type of the UDF argument at position 3.
+   * @tparam A4 type of the UDF argument at position 4.
+   * @tparam A5 type of the UDF argument at position 5.
+   * @tparam A6 type of the UDF argument at position 6.
+   * @tparam A7 type of the UDF argument at position 7.
+   * @tparam A8 type of the UDF argument at position 8.
+   * @tparam A9 type of the UDF argument at position 9.
+   * @tparam A10 type of the UDF argument at position 10.
+   * @tparam A11 type of the UDF argument at position 11.
+   * @tparam A12 type of the UDF argument at position 12.
+   * @tparam A13 type of the UDF argument at position 13.
+   * @tparam A14 type of the UDF argument at position 14.
+   * @tparam A15 type of the UDF argument at position 15.
+   * @tparam A16 type of the UDF argument at position 16.
+   * @tparam A17 type of the UDF argument at position 17.
+   * @tparam A18 type of the UDF argument at position 18.
+   * @tparam A19 type of the UDF argument at position 19.
+   * @tparam A20 type of the UDF argument at position 20.
+   * @tparam A21 type of the UDF argument at position 21.
+   */
+  def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: 
TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: 
TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, 
A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag, A21: 
TypeTag](name: String, func: Function21[A1, A2, A3, A4, A5, A6, A7, A8, A9, 
A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, RT]): Unit = {
+    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[RT].dataType, e)
     functionRegistry.registerFunction(name, builder)
   }
 
-  def registerFunction[T: TypeTag](name: String, func: Function22[_, _, _, _, 
_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = {
-    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[T].dataType, e)
+  /**
+   * Register a Scala closure of 22 arguments as user-defined function (UDF).
+   * @tparam RT return type of UDF.
+   * @tparam A1 type of the UDF argument at position 1.
+   * @tparam A2 type of the UDF argument at position 2.
+   * @tparam A3 type of the UDF argument at position 3.
+   * @tparam A4 type of the UDF argument at position 4.
+   * @tparam A5 type of the UDF argument at position 5.
+   * @tparam A6 type of the UDF argument at position 6.
+   * @tparam A7 type of the UDF argument at position 7.
+   * @tparam A8 type of the UDF argument at position 8.
+   * @tparam A9 type of the UDF argument at position 9.
+   * @tparam A10 type of the UDF argument at position 10.
+   * @tparam A11 type of the UDF argument at position 11.
+   * @tparam A12 type of the UDF argument at position 12.
+   * @tparam A13 type of the UDF argument at position 13.
+   * @tparam A14 type of the UDF argument at position 14.
+   * @tparam A15 type of the UDF argument at position 15.
+   * @tparam A16 type of the UDF argument at position 16.
+   * @tparam A17 type of the UDF argument at position 17.
+   * @tparam A18 type of the UDF argument at position 18.
+   * @tparam A19 type of the UDF argument at position 19.
+   * @tparam A20 type of the UDF argument at position 20.
+   * @tparam A21 type of the UDF argument at position 21.
+   * @tparam A22 type of the UDF argument at position 22.
+   */
+  def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: 
TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: 
TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, 
A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag, A21: 
TypeTag, A22: TypeTag](name: String, func: Function22[A1, A2, A3, A4, A5, A6, 
A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, A22, 
RT]): Unit = {
+    def builder(e: Seq[Expression]) = ScalaUdf(func, 
ScalaReflection.schemaFor[RT].dataType, e)
     functionRegistry.registerFunction(name, builder)
   }
+
+  /**
+   * Register a user-defined function with 1 arguments.
+   */
+  def register(name: String, f: UDF1[_, _], returnType: DataType) = {
+    functionRegistry.registerFunction(
+      name,
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF1[Any, Any]].call(_: 
Any), returnType, e))
+  }
+
+  /**
+   * Register a user-defined function with 2 arguments.
+   */
+  def register(name: String, f: UDF2[_, _, _], returnType: DataType) = {
+    functionRegistry.registerFunction(
+      name,
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF2[Any, Any, 
Any]].call(_: Any, _: Any), returnType, e))
+  }
+
+  /**
+   * Register a user-defined function with 3 arguments.
+   */
+  def register(name: String, f: UDF3[_, _, _, _], returnType: DataType) = {
+    functionRegistry.registerFunction(
+      name,
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF3[Any, Any, Any, 
Any]].call(_: Any, _: Any, _: Any), returnType, e))
+  }
+
+  /**
+   * Register a user-defined function with 4 arguments.
+   */
+  def register(name: String, f: UDF4[_, _, _, _, _], returnType: DataType) = {
+    functionRegistry.registerFunction(
+      name,
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF4[Any, Any, Any, Any, 
Any]].call(_: Any, _: Any, _: Any, _: Any), returnType, e))
+  }
+
+  /**
+   * Register a user-defined function with 5 arguments.
+   */
+  def register(name: String, f: UDF5[_, _, _, _, _, _], returnType: DataType) 
= {
+    functionRegistry.registerFunction(
+      name,
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF5[Any, Any, Any, Any, 
Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any), returnType, e))
+  }
+
+  /**
+   * Register a user-defined function with 6 arguments.
+   */
+  def register(name: String, f: UDF6[_, _, _, _, _, _, _], returnType: 
DataType) = {
+    functionRegistry.registerFunction(
+      name,
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF6[Any, Any, Any, Any, 
Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any), 
returnType, e))
+  }
+
+  /**
+   * Register a user-defined function with 7 arguments.
+   */
+  def register(name: String, f: UDF7[_, _, _, _, _, _, _, _], returnType: 
DataType) = {
+    functionRegistry.registerFunction(
+      name,
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF7[Any, Any, Any, Any, 
Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: 
Any), returnType, e))
+  }
+
+  /**
+   * Register a user-defined function with 8 arguments.
+   */
+  def register(name: String, f: UDF8[_, _, _, _, _, _, _, _, _], returnType: 
DataType) = {
+    functionRegistry.registerFunction(
+      name,
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF8[Any, Any, Any, Any, 
Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, 
_: Any, _: Any), returnType, e))
+  }
+
+  /**
+   * Register a user-defined function with 9 arguments.
+   */
+  def register(name: String, f: UDF9[_, _, _, _, _, _, _, _, _, _], 
returnType: DataType) = {
+    functionRegistry.registerFunction(
+      name,
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF9[Any, Any, Any, Any, 
Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: 
Any, _: Any, _: Any, _: Any), returnType, e))
+  }
+
+  /**
+   * Register a user-defined function with 10 arguments.
+   */
+  def register(name: String, f: UDF10[_, _, _, _, _, _, _, _, _, _, _], 
returnType: DataType) = {
+    functionRegistry.registerFunction(
+      name,
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF10[Any, Any, Any, 
Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, 
_: Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e))
+  }
+
+  /**
+   * Register a user-defined function with 11 arguments.
+   */
+  def register(name: String, f: UDF11[_, _, _, _, _, _, _, _, _, _, _, _], 
returnType: DataType) = {
+    functionRegistry.registerFunction(
+      name,
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF11[Any, Any, Any, 
Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: 
Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e))
+  }
+
+  /**
+   * Register a user-defined function with 12 arguments.
+   */
+  def register(name: String, f: UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _], 
returnType: DataType) = {
+    functionRegistry.registerFunction(
+      name,
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF12[Any, Any, Any, 
Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, 
_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), 
returnType, e))
+  }
+
+  /**
+   * Register a user-defined function with 13 arguments.
+   */
+  def register(name: String, f: UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, 
_], returnType: DataType) = {
+    functionRegistry.registerFunction(
+      name,
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF13[Any, Any, Any, 
Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: 
Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: 
Any), returnType, e))
+  }
+
+  /**
+   * Register a user-defined function with 14 arguments.
+   */
+  def register(name: String, f: UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, 
_, _], returnType: DataType) = {
+    functionRegistry.registerFunction(
+      name,
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF14[Any, Any, Any, 
Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: 
Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: 
Any, _: Any, _: Any), returnType, e))
+  }
+
+  /**
+   * Register a user-defined function with 15 arguments.
+   */
+  def register(name: String, f: UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, 
_, _, _], returnType: DataType) = {
+    functionRegistry.registerFunction(
+      name,
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF15[Any, Any, Any, 
Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, 
_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, 
_: Any, _: Any, _: Any, _: Any), returnType, e))
+  }
+
+  /**
+   * Register a user-defined function with 16 arguments.
+   */
+  def register(name: String, f: UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, 
_, _, _, _], returnType: DataType) = {
+    functionRegistry.registerFunction(
+      name,
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF16[Any, Any, Any, 
Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: 
Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: 
Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e))
+  }
+
+  /**
+   * Register a user-defined function with 17 arguments.
+   */
+  def register(name: String, f: UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, 
_, _, _, _, _], returnType: DataType) = {
+    functionRegistry.registerFunction(
+      name,
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF17[Any, Any, Any, 
Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, 
Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: 
Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), 
returnType, e))
+  }
+
+  /**
+   * Register a user-defined function with 18 arguments.
+   */
+  def register(name: String, f: UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, 
_, _, _, _, _, _], returnType: DataType) = {
+    functionRegistry.registerFunction(
+      name,
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF18[Any, Any, Any, 
Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, 
Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: 
Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), 
returnType, e))
+  }
+
+  /**
+   * Register a user-defined function with 19 arguments.
+   */
+  def register(name: String, f: UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, 
_, _, _, _, _, _, _], returnType: DataType) = {
+    functionRegistry.registerFunction(
+      name,
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF19[Any, Any, Any, 
Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, 
Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: 
Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: 
Any), returnType, e))
+  }
+
+  /**
+   * Register a user-defined function with 20 arguments.
+   */
+  def register(name: String, f: UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, 
_, _, _, _, _, _, _, _], returnType: DataType) = {
+    functionRegistry.registerFunction(
+      name,
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF20[Any, Any, Any, 
Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, 
Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, 
_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, 
_: Any, _: Any), returnType, e))
+  }
+
+  /**
+   * Register a user-defined function with 21 arguments.
+   */
+  def register(name: String, f: UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, 
_, _, _, _, _, _, _, _, _], returnType: DataType) = {
+    functionRegistry.registerFunction(
+      name,
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF21[Any, Any, Any, 
Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, 
Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: 
Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: 
Any, _: Any, _: Any, _: Any), returnType, e))
+  }
+
+  /**
+   * Register a user-defined function with 22 arguments.
+   */
+  def register(name: String, f: UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, 
_, _, _, _, _, _, _, _, _, _], returnType: DataType) = {
+    functionRegistry.registerFunction(
+      name,
+      (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF22[Any, Any, Any, 
Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, 
Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: 
Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: 
Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e))
+  }
+
   // scalastyle:on
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/1881431d/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
index cbdb3e6..6c95bad 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
@@ -766,7 +766,7 @@ class SQLQuerySuite extends QueryTest with 
BeforeAndAfterAll {
   }
 
   test("SPARK-3371 Renaming a function expression with group by gives error") {
-    registerFunction("len", (s: String) => s.length)
+    udf.register("len", (s: String) => s.length)
     checkAnswer(
       sql("SELECT len(value) as temp FROM testData WHERE key = 1 group by 
len(value)"), 1)
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/1881431d/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala
index 720953a..0c98120 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala
@@ -27,23 +27,22 @@ case class FunctionResult(f1: String, f2: String)
 class UDFSuite extends QueryTest {
 
   test("Simple UDF") {
-    registerFunction("strLenScala", (_: String).length)
+    udf.register("strLenScala", (_: String).length)
     assert(sql("SELECT strLenScala('test')").first().getInt(0) === 4)
   }
 
   test("ZeroArgument UDF") {
-    registerFunction("random0", () => { Math.random()})
+    udf.register("random0", () => { Math.random()})
     assert(sql("SELECT random0()").first().getDouble(0) >= 0.0)
   }
 
   test("TwoArgument UDF") {
-    registerFunction("strLenScala", (_: String).length + (_:Int))
+    udf.register("strLenScala", (_: String).length + (_:Int))
     assert(sql("SELECT strLenScala('test', 1)").first().getInt(0) === 5)
   }
 
-
   test("struct UDF") {
-    registerFunction("returnStruct", (f1: String, f2: String) => 
FunctionResult(f1, f2))
+    udf.register("returnStruct", (f1: String, f2: String) => 
FunctionResult(f1, f2))
 
     val result=
       sql("SELECT returnStruct('test', 'test2') as ret")

http://git-wip-us.apache.org/repos/asf/spark/blob/1881431d/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala
index a0d54d1..fbc8704 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala
@@ -81,7 +81,7 @@ class UserDefinedTypeSuite extends QueryTest {
   }
 
   test("UDTs and UDFs") {
-    registerFunction("testType", (d: MyDenseVector) => 
d.isInstanceOf[MyDenseVector])
+    udf.register("testType", (d: MyDenseVector) => 
d.isInstanceOf[MyDenseVector])
     pointsRDD.registerTempTable("points")
     checkAnswer(
       sql("SELECT testType(features) from points"),

http://git-wip-us.apache.org/repos/asf/spark/blob/1881431d/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala
index 5fc8d8d..5dafcd6 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala
@@ -50,7 +50,7 @@ class HiveUdfSuite extends QueryTest {
   import TestHive._
 
   test("spark sql udf test that returns a struct") {
-    registerFunction("getStruct", (_: Int) => Fields(1, 2, 3, 4, 5))
+    udf.register("getStruct", (_: Int) => Fields(1, 2, 3, 4, 5))
     assert(sql(
       """
         |SELECT getStruct(1).f1,


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to