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