aboutsummaryrefslogtreecommitdiff
path: root/sql
diff options
context:
space:
mode:
authorReynold Xin <rxin@databricks.com>2015-01-15 16:15:12 -0800
committerReynold Xin <rxin@databricks.com>2015-01-15 16:15:12 -0800
commit1881431dd50e93a6948e4966d33742727f27e917 (patch)
tree012aa377cb3d891ce563f5225407176b55936081 /sql
parent3c8650c12ad7a97852e7bd76153210493fd83e92 (diff)
downloadspark-1881431dd50e93a6948e4966d33742727f27e917.tar.gz
spark-1881431dd50e93a6948e4966d33742727f27e917.tar.bz2
spark-1881431dd50e93a6948e4966d33742727f27e917.zip
[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 <rxin@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.
Diffstat (limited to 'sql')
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala29
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala692
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala2
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala9
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala2
-rw-r--r--sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala2
6 files changed, 666 insertions, 70 deletions
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 279671ced0..8ad1753dab 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
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 5fb472686c..2e9d037f93 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
}
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 cbdb3e64bb..6c95bad697 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)
}
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 720953ae37..0c98120031 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")
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 a0d54d17f5..fbc8704f78 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"),
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 5fc8d8dbe3..5dafcd6c0a 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,