1/*
2 * Licensed to the Apache Software Foundation (ASF) under one or more
3 * contributor license agreements.  See the NOTICE file distributed with
4 * this work for additional information regarding copyright ownership.
5 * The ASF licenses this file to You under the Apache License, Version 2.0
6 * (the "License"); you may not use this file except in compliance with
7 * the License.  You may obtain a copy of the License at
8 *
9 *    http://www.apache.org/licenses/LICENSE-2.0
10 *
11 * Unless required by applicable law or agreed to in writing, software
12 * distributed under the License is distributed on an "AS IS" BASIS,
13 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 * See the License for the specific language governing permissions and
15 * limitations under the License.
16 */
17
18package org.apache.spark.sql
19
20import java.io.IOException
21import java.lang.reflect.{ParameterizedType, Type}
22
23import scala.reflect.runtime.universe.TypeTag
24import scala.util.Try
25
26import org.apache.spark.annotation.InterfaceStability
27import org.apache.spark.internal.Logging
28import org.apache.spark.sql.api.java._
29import org.apache.spark.sql.catalyst.{JavaTypeInference, ScalaReflection}
30import org.apache.spark.sql.catalyst.analysis.FunctionRegistry
31import org.apache.spark.sql.catalyst.expressions.{Expression, ScalaUDF}
32import org.apache.spark.sql.execution.aggregate.ScalaUDAF
33import org.apache.spark.sql.execution.python.UserDefinedPythonFunction
34import org.apache.spark.sql.expressions.{UserDefinedAggregateFunction, UserDefinedFunction}
35import org.apache.spark.sql.types.{DataType, DataTypes}
36import org.apache.spark.util.Utils
37
38/**
39 * Functions for registering user-defined functions. Use `SparkSession.udf` to access this:
40 *
41 * {{{
42 *   spark.udf
43 * }}}
44 *
45 * @note The user-defined functions must be deterministic.
46 *
47 * @since 1.3.0
48 */
49@InterfaceStability.Stable
50class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends Logging {
51
52  protected[sql] def registerPython(name: String, udf: UserDefinedPythonFunction): Unit = {
53    log.debug(
54      s"""
55        | Registering new PythonUDF:
56        | name: $name
57        | command: ${udf.func.command.toSeq}
58        | envVars: ${udf.func.envVars}
59        | pythonIncludes: ${udf.func.pythonIncludes}
60        | pythonExec: ${udf.func.pythonExec}
61        | dataType: ${udf.dataType}
62      """.stripMargin)
63
64    functionRegistry.registerFunction(name, udf.builder)
65  }
66
67  /**
68   * Register a user-defined aggregate function (UDAF).
69   *
70   * @param name the name of the UDAF.
71   * @param udaf the UDAF needs to be registered.
72   * @return the registered UDAF.
73   */
74  def register(
75      name: String,
76      udaf: UserDefinedAggregateFunction): UserDefinedAggregateFunction = {
77    def builder(children: Seq[Expression]) = ScalaUDAF(children, udaf)
78    functionRegistry.registerFunction(name, builder)
79    udaf
80  }
81
82  // scalastyle:off line.size.limit
83
84  /* register 0-22 were generated by this script
85
86    (0 to 22).map { x =>
87      val types = (1 to x).foldRight("RT")((i, s) => {s"A$i, $s"})
88      val typeTags = (1 to x).map(i => s"A${i}: TypeTag").foldLeft("RT: TypeTag")(_ + ", " + _)
89      val inputTypes = (1 to x).foldRight("Nil")((i, s) => {s"ScalaReflection.schemaFor[A$i].dataType :: $s"})
90      println(s"""
91        /**
92         * Register a Scala closure of ${x} arguments as user-defined function (UDF).
93         * @tparam RT return type of UDF.
94         * @since 1.3.0
95         */
96        def register[$typeTags](name: String, func: Function$x[$types]): UserDefinedFunction = {
97          val dataType = ScalaReflection.schemaFor[RT].dataType
98          val inputTypes = Try($inputTypes).toOption
99          def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name))
100          functionRegistry.registerFunction(name, builder)
101          UserDefinedFunction(func, dataType, inputTypes)
102        }""")
103    }
104
105    (1 to 22).foreach { i =>
106      val extTypeArgs = (1 to i).map(_ => "_").mkString(", ")
107      val anyTypeArgs = (1 to i).map(_ => "Any").mkString(", ")
108      val anyCast = s".asInstanceOf[UDF$i[$anyTypeArgs, Any]]"
109      val anyParams = (1 to i).map(_ => "_: Any").mkString(", ")
110      println(s"""
111         |/**
112         | * Register a user-defined function with ${i} arguments.
113         | * @since 1.3.0
114         | */
115         |def register(name: String, f: UDF$i[$extTypeArgs, _], returnType: DataType): Unit = {
116         |  val func = f$anyCast.call($anyParams)
117         |  functionRegistry.registerFunction(
118         |    name,
119         |    (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
120         |}""".stripMargin)
121    }
122    */
123
124  /**
125   * Register a Scala closure of 0 arguments as user-defined function (UDF).
126   * @tparam RT return type of UDF.
127   * @since 1.3.0
128   */
129  def register[RT: TypeTag](name: String, func: Function0[RT]): UserDefinedFunction = {
130    val dataType = ScalaReflection.schemaFor[RT].dataType
131    val inputTypes = Try(Nil).toOption
132    def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name))
133    functionRegistry.registerFunction(name, builder)
134    UserDefinedFunction(func, dataType, inputTypes)
135  }
136
137  /**
138   * Register a Scala closure of 1 arguments as user-defined function (UDF).
139   * @tparam RT return type of UDF.
140   * @since 1.3.0
141   */
142  def register[RT: TypeTag, A1: TypeTag](name: String, func: Function1[A1, RT]): UserDefinedFunction = {
143    val dataType = ScalaReflection.schemaFor[RT].dataType
144    val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: Nil).toOption
145    def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name))
146    functionRegistry.registerFunction(name, builder)
147    UserDefinedFunction(func, dataType, inputTypes)
148  }
149
150  /**
151   * Register a Scala closure of 2 arguments as user-defined function (UDF).
152   * @tparam RT return type of UDF.
153   * @since 1.3.0
154   */
155  def register[RT: TypeTag, A1: TypeTag, A2: TypeTag](name: String, func: Function2[A1, A2, RT]): UserDefinedFunction = {
156    val dataType = ScalaReflection.schemaFor[RT].dataType
157    val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: Nil).toOption
158    def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name))
159    functionRegistry.registerFunction(name, builder)
160    UserDefinedFunction(func, dataType, inputTypes)
161  }
162
163  /**
164   * Register a Scala closure of 3 arguments as user-defined function (UDF).
165   * @tparam RT return type of UDF.
166   * @since 1.3.0
167   */
168  def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag](name: String, func: Function3[A1, A2, A3, RT]): UserDefinedFunction = {
169    val dataType = ScalaReflection.schemaFor[RT].dataType
170    val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: Nil).toOption
171    def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name))
172    functionRegistry.registerFunction(name, builder)
173    UserDefinedFunction(func, dataType, inputTypes)
174  }
175
176  /**
177   * Register a Scala closure of 4 arguments as user-defined function (UDF).
178   * @tparam RT return type of UDF.
179   * @since 1.3.0
180   */
181  def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag](name: String, func: Function4[A1, A2, A3, A4, RT]): UserDefinedFunction = {
182    val dataType = ScalaReflection.schemaFor[RT].dataType
183    val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: Nil).toOption
184    def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name))
185    functionRegistry.registerFunction(name, builder)
186    UserDefinedFunction(func, dataType, inputTypes)
187  }
188
189  /**
190   * Register a Scala closure of 5 arguments as user-defined function (UDF).
191   * @tparam RT return type of UDF.
192   * @since 1.3.0
193   */
194  def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag](name: String, func: Function5[A1, A2, A3, A4, A5, RT]): UserDefinedFunction = {
195    val dataType = ScalaReflection.schemaFor[RT].dataType
196    val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: ScalaReflection.schemaFor[A5].dataType :: Nil).toOption
197    def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name))
198    functionRegistry.registerFunction(name, builder)
199    UserDefinedFunction(func, dataType, inputTypes)
200  }
201
202  /**
203   * Register a Scala closure of 6 arguments as user-defined function (UDF).
204   * @tparam RT return type of UDF.
205   * @since 1.3.0
206   */
207  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]): UserDefinedFunction = {
208    val dataType = ScalaReflection.schemaFor[RT].dataType
209    val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: ScalaReflection.schemaFor[A5].dataType :: ScalaReflection.schemaFor[A6].dataType :: Nil).toOption
210    def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name))
211    functionRegistry.registerFunction(name, builder)
212    UserDefinedFunction(func, dataType, inputTypes)
213  }
214
215  /**
216   * Register a Scala closure of 7 arguments as user-defined function (UDF).
217   * @tparam RT return type of UDF.
218   * @since 1.3.0
219   */
220  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]): UserDefinedFunction = {
221    val dataType = ScalaReflection.schemaFor[RT].dataType
222    val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: ScalaReflection.schemaFor[A5].dataType :: ScalaReflection.schemaFor[A6].dataType :: ScalaReflection.schemaFor[A7].dataType :: Nil).toOption
223    def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name))
224    functionRegistry.registerFunction(name, builder)
225    UserDefinedFunction(func, dataType, inputTypes)
226  }
227
228  /**
229   * Register a Scala closure of 8 arguments as user-defined function (UDF).
230   * @tparam RT return type of UDF.
231   * @since 1.3.0
232   */
233  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]): UserDefinedFunction = {
234    val dataType = ScalaReflection.schemaFor[RT].dataType
235    val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: ScalaReflection.schemaFor[A5].dataType :: ScalaReflection.schemaFor[A6].dataType :: ScalaReflection.schemaFor[A7].dataType :: ScalaReflection.schemaFor[A8].dataType :: Nil).toOption
236    def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name))
237    functionRegistry.registerFunction(name, builder)
238    UserDefinedFunction(func, dataType, inputTypes)
239  }
240
241  /**
242   * Register a Scala closure of 9 arguments as user-defined function (UDF).
243   * @tparam RT return type of UDF.
244   * @since 1.3.0
245   */
246  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]): UserDefinedFunction = {
247    val dataType = ScalaReflection.schemaFor[RT].dataType
248    val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: ScalaReflection.schemaFor[A5].dataType :: ScalaReflection.schemaFor[A6].dataType :: ScalaReflection.schemaFor[A7].dataType :: ScalaReflection.schemaFor[A8].dataType :: ScalaReflection.schemaFor[A9].dataType :: Nil).toOption
249    def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name))
250    functionRegistry.registerFunction(name, builder)
251    UserDefinedFunction(func, dataType, inputTypes)
252  }
253
254  /**
255   * Register a Scala closure of 10 arguments as user-defined function (UDF).
256   * @tparam RT return type of UDF.
257   * @since 1.3.0
258   */
259  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]): UserDefinedFunction = {
260    val dataType = ScalaReflection.schemaFor[RT].dataType
261    val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: ScalaReflection.schemaFor[A5].dataType :: ScalaReflection.schemaFor[A6].dataType :: ScalaReflection.schemaFor[A7].dataType :: ScalaReflection.schemaFor[A8].dataType :: ScalaReflection.schemaFor[A9].dataType :: ScalaReflection.schemaFor[A10].dataType :: Nil).toOption
262    def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name))
263    functionRegistry.registerFunction(name, builder)
264    UserDefinedFunction(func, dataType, inputTypes)
265  }
266
267  /**
268   * Register a Scala closure of 11 arguments as user-defined function (UDF).
269   * @tparam RT return type of UDF.
270   * @since 1.3.0
271   */
272  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]): UserDefinedFunction = {
273    val dataType = ScalaReflection.schemaFor[RT].dataType
274    val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: ScalaReflection.schemaFor[A5].dataType :: ScalaReflection.schemaFor[A6].dataType :: ScalaReflection.schemaFor[A7].dataType :: ScalaReflection.schemaFor[A8].dataType :: ScalaReflection.schemaFor[A9].dataType :: ScalaReflection.schemaFor[A10].dataType :: ScalaReflection.schemaFor[A11].dataType :: Nil).toOption
275    def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name))
276    functionRegistry.registerFunction(name, builder)
277    UserDefinedFunction(func, dataType, inputTypes)
278  }
279
280  /**
281   * Register a Scala closure of 12 arguments as user-defined function (UDF).
282   * @tparam RT return type of UDF.
283   * @since 1.3.0
284   */
285  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]): UserDefinedFunction = {
286    val dataType = ScalaReflection.schemaFor[RT].dataType
287    val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: ScalaReflection.schemaFor[A5].dataType :: ScalaReflection.schemaFor[A6].dataType :: ScalaReflection.schemaFor[A7].dataType :: ScalaReflection.schemaFor[A8].dataType :: ScalaReflection.schemaFor[A9].dataType :: ScalaReflection.schemaFor[A10].dataType :: ScalaReflection.schemaFor[A11].dataType :: ScalaReflection.schemaFor[A12].dataType :: Nil).toOption
288    def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name))
289    functionRegistry.registerFunction(name, builder)
290    UserDefinedFunction(func, dataType, inputTypes)
291  }
292
293  /**
294   * Register a Scala closure of 13 arguments as user-defined function (UDF).
295   * @tparam RT return type of UDF.
296   * @since 1.3.0
297   */
298  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]): UserDefinedFunction = {
299    val dataType = ScalaReflection.schemaFor[RT].dataType
300    val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: ScalaReflection.schemaFor[A5].dataType :: ScalaReflection.schemaFor[A6].dataType :: ScalaReflection.schemaFor[A7].dataType :: ScalaReflection.schemaFor[A8].dataType :: ScalaReflection.schemaFor[A9].dataType :: ScalaReflection.schemaFor[A10].dataType :: ScalaReflection.schemaFor[A11].dataType :: ScalaReflection.schemaFor[A12].dataType :: ScalaReflection.schemaFor[A13].dataType :: Nil).toOption
301    def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name))
302    functionRegistry.registerFunction(name, builder)
303    UserDefinedFunction(func, dataType, inputTypes)
304  }
305
306  /**
307   * Register a Scala closure of 14 arguments as user-defined function (UDF).
308   * @tparam RT return type of UDF.
309   * @since 1.3.0
310   */
311  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]): UserDefinedFunction = {
312    val dataType = ScalaReflection.schemaFor[RT].dataType
313    val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: ScalaReflection.schemaFor[A5].dataType :: ScalaReflection.schemaFor[A6].dataType :: ScalaReflection.schemaFor[A7].dataType :: ScalaReflection.schemaFor[A8].dataType :: ScalaReflection.schemaFor[A9].dataType :: ScalaReflection.schemaFor[A10].dataType :: ScalaReflection.schemaFor[A11].dataType :: ScalaReflection.schemaFor[A12].dataType :: ScalaReflection.schemaFor[A13].dataType :: ScalaReflection.schemaFor[A14].dataType :: Nil).toOption
314    def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name))
315    functionRegistry.registerFunction(name, builder)
316    UserDefinedFunction(func, dataType, inputTypes)
317  }
318
319  /**
320   * Register a Scala closure of 15 arguments as user-defined function (UDF).
321   * @tparam RT return type of UDF.
322   * @since 1.3.0
323   */
324  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]): UserDefinedFunction = {
325    val dataType = ScalaReflection.schemaFor[RT].dataType
326    val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: ScalaReflection.schemaFor[A5].dataType :: ScalaReflection.schemaFor[A6].dataType :: ScalaReflection.schemaFor[A7].dataType :: ScalaReflection.schemaFor[A8].dataType :: ScalaReflection.schemaFor[A9].dataType :: ScalaReflection.schemaFor[A10].dataType :: ScalaReflection.schemaFor[A11].dataType :: ScalaReflection.schemaFor[A12].dataType :: ScalaReflection.schemaFor[A13].dataType :: ScalaReflection.schemaFor[A14].dataType :: ScalaReflection.schemaFor[A15].dataType :: Nil).toOption
327    def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name))
328    functionRegistry.registerFunction(name, builder)
329    UserDefinedFunction(func, dataType, inputTypes)
330  }
331
332  /**
333   * Register a Scala closure of 16 arguments as user-defined function (UDF).
334   * @tparam RT return type of UDF.
335   * @since 1.3.0
336   */
337  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]): UserDefinedFunction = {
338    val dataType = ScalaReflection.schemaFor[RT].dataType
339    val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: ScalaReflection.schemaFor[A5].dataType :: ScalaReflection.schemaFor[A6].dataType :: ScalaReflection.schemaFor[A7].dataType :: ScalaReflection.schemaFor[A8].dataType :: ScalaReflection.schemaFor[A9].dataType :: ScalaReflection.schemaFor[A10].dataType :: ScalaReflection.schemaFor[A11].dataType :: ScalaReflection.schemaFor[A12].dataType :: ScalaReflection.schemaFor[A13].dataType :: ScalaReflection.schemaFor[A14].dataType :: ScalaReflection.schemaFor[A15].dataType :: ScalaReflection.schemaFor[A16].dataType :: Nil).toOption
340    def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name))
341    functionRegistry.registerFunction(name, builder)
342    UserDefinedFunction(func, dataType, inputTypes)
343  }
344
345  /**
346   * Register a Scala closure of 17 arguments as user-defined function (UDF).
347   * @tparam RT return type of UDF.
348   * @since 1.3.0
349   */
350  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]): UserDefinedFunction = {
351    val dataType = ScalaReflection.schemaFor[RT].dataType
352    val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: ScalaReflection.schemaFor[A5].dataType :: ScalaReflection.schemaFor[A6].dataType :: ScalaReflection.schemaFor[A7].dataType :: ScalaReflection.schemaFor[A8].dataType :: ScalaReflection.schemaFor[A9].dataType :: ScalaReflection.schemaFor[A10].dataType :: ScalaReflection.schemaFor[A11].dataType :: ScalaReflection.schemaFor[A12].dataType :: ScalaReflection.schemaFor[A13].dataType :: ScalaReflection.schemaFor[A14].dataType :: ScalaReflection.schemaFor[A15].dataType :: ScalaReflection.schemaFor[A16].dataType :: ScalaReflection.schemaFor[A17].dataType :: Nil).toOption
353    def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name))
354    functionRegistry.registerFunction(name, builder)
355    UserDefinedFunction(func, dataType, inputTypes)
356  }
357
358  /**
359   * Register a Scala closure of 18 arguments as user-defined function (UDF).
360   * @tparam RT return type of UDF.
361   * @since 1.3.0
362   */
363  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]): UserDefinedFunction = {
364    val dataType = ScalaReflection.schemaFor[RT].dataType
365    val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: ScalaReflection.schemaFor[A5].dataType :: ScalaReflection.schemaFor[A6].dataType :: ScalaReflection.schemaFor[A7].dataType :: ScalaReflection.schemaFor[A8].dataType :: ScalaReflection.schemaFor[A9].dataType :: ScalaReflection.schemaFor[A10].dataType :: ScalaReflection.schemaFor[A11].dataType :: ScalaReflection.schemaFor[A12].dataType :: ScalaReflection.schemaFor[A13].dataType :: ScalaReflection.schemaFor[A14].dataType :: ScalaReflection.schemaFor[A15].dataType :: ScalaReflection.schemaFor[A16].dataType :: ScalaReflection.schemaFor[A17].dataType :: ScalaReflection.schemaFor[A18].dataType :: Nil).toOption
366    def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name))
367    functionRegistry.registerFunction(name, builder)
368    UserDefinedFunction(func, dataType, inputTypes)
369  }
370
371  /**
372   * Register a Scala closure of 19 arguments as user-defined function (UDF).
373   * @tparam RT return type of UDF.
374   * @since 1.3.0
375   */
376  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]): UserDefinedFunction = {
377    val dataType = ScalaReflection.schemaFor[RT].dataType
378    val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: ScalaReflection.schemaFor[A5].dataType :: ScalaReflection.schemaFor[A6].dataType :: ScalaReflection.schemaFor[A7].dataType :: ScalaReflection.schemaFor[A8].dataType :: ScalaReflection.schemaFor[A9].dataType :: ScalaReflection.schemaFor[A10].dataType :: ScalaReflection.schemaFor[A11].dataType :: ScalaReflection.schemaFor[A12].dataType :: ScalaReflection.schemaFor[A13].dataType :: ScalaReflection.schemaFor[A14].dataType :: ScalaReflection.schemaFor[A15].dataType :: ScalaReflection.schemaFor[A16].dataType :: ScalaReflection.schemaFor[A17].dataType :: ScalaReflection.schemaFor[A18].dataType :: ScalaReflection.schemaFor[A19].dataType :: Nil).toOption
379    def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name))
380    functionRegistry.registerFunction(name, builder)
381    UserDefinedFunction(func, dataType, inputTypes)
382  }
383
384  /**
385   * Register a Scala closure of 20 arguments as user-defined function (UDF).
386   * @tparam RT return type of UDF.
387   * @since 1.3.0
388   */
389  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]): UserDefinedFunction = {
390    val dataType = ScalaReflection.schemaFor[RT].dataType
391    val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: ScalaReflection.schemaFor[A5].dataType :: ScalaReflection.schemaFor[A6].dataType :: ScalaReflection.schemaFor[A7].dataType :: ScalaReflection.schemaFor[A8].dataType :: ScalaReflection.schemaFor[A9].dataType :: ScalaReflection.schemaFor[A10].dataType :: ScalaReflection.schemaFor[A11].dataType :: ScalaReflection.schemaFor[A12].dataType :: ScalaReflection.schemaFor[A13].dataType :: ScalaReflection.schemaFor[A14].dataType :: ScalaReflection.schemaFor[A15].dataType :: ScalaReflection.schemaFor[A16].dataType :: ScalaReflection.schemaFor[A17].dataType :: ScalaReflection.schemaFor[A18].dataType :: ScalaReflection.schemaFor[A19].dataType :: ScalaReflection.schemaFor[A20].dataType :: Nil).toOption
392    def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name))
393    functionRegistry.registerFunction(name, builder)
394    UserDefinedFunction(func, dataType, inputTypes)
395  }
396
397  /**
398   * Register a Scala closure of 21 arguments as user-defined function (UDF).
399   * @tparam RT return type of UDF.
400   * @since 1.3.0
401   */
402  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]): UserDefinedFunction = {
403    val dataType = ScalaReflection.schemaFor[RT].dataType
404    val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: ScalaReflection.schemaFor[A5].dataType :: ScalaReflection.schemaFor[A6].dataType :: ScalaReflection.schemaFor[A7].dataType :: ScalaReflection.schemaFor[A8].dataType :: ScalaReflection.schemaFor[A9].dataType :: ScalaReflection.schemaFor[A10].dataType :: ScalaReflection.schemaFor[A11].dataType :: ScalaReflection.schemaFor[A12].dataType :: ScalaReflection.schemaFor[A13].dataType :: ScalaReflection.schemaFor[A14].dataType :: ScalaReflection.schemaFor[A15].dataType :: ScalaReflection.schemaFor[A16].dataType :: ScalaReflection.schemaFor[A17].dataType :: ScalaReflection.schemaFor[A18].dataType :: ScalaReflection.schemaFor[A19].dataType :: ScalaReflection.schemaFor[A20].dataType :: ScalaReflection.schemaFor[A21].dataType :: Nil).toOption
405    def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name))
406    functionRegistry.registerFunction(name, builder)
407    UserDefinedFunction(func, dataType, inputTypes)
408  }
409
410  /**
411   * Register a Scala closure of 22 arguments as user-defined function (UDF).
412   * @tparam RT return type of UDF.
413   * @since 1.3.0
414   */
415  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]): UserDefinedFunction = {
416    val dataType = ScalaReflection.schemaFor[RT].dataType
417    val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: ScalaReflection.schemaFor[A5].dataType :: ScalaReflection.schemaFor[A6].dataType :: ScalaReflection.schemaFor[A7].dataType :: ScalaReflection.schemaFor[A8].dataType :: ScalaReflection.schemaFor[A9].dataType :: ScalaReflection.schemaFor[A10].dataType :: ScalaReflection.schemaFor[A11].dataType :: ScalaReflection.schemaFor[A12].dataType :: ScalaReflection.schemaFor[A13].dataType :: ScalaReflection.schemaFor[A14].dataType :: ScalaReflection.schemaFor[A15].dataType :: ScalaReflection.schemaFor[A16].dataType :: ScalaReflection.schemaFor[A17].dataType :: ScalaReflection.schemaFor[A18].dataType :: ScalaReflection.schemaFor[A19].dataType :: ScalaReflection.schemaFor[A20].dataType :: ScalaReflection.schemaFor[A21].dataType :: ScalaReflection.schemaFor[A22].dataType :: Nil).toOption
418    def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name))
419    functionRegistry.registerFunction(name, builder)
420    UserDefinedFunction(func, dataType, inputTypes)
421  }
422
423  //////////////////////////////////////////////////////////////////////////////////////////////
424  //////////////////////////////////////////////////////////////////////////////////////////////
425
426  /**
427   * Register a Java UDF class using reflection, for use from pyspark
428   *
429   * @param name   udf name
430   * @param className   fully qualified class name of udf
431   * @param returnDataType  return type of udf. If it is null, spark would try to infer
432   *                        via reflection.
433   */
434  private[sql] def registerJava(name: String, className: String, returnDataType: DataType): Unit = {
435
436    try {
437      val clazz = Utils.classForName(className)
438      val udfInterfaces = clazz.getGenericInterfaces
439        .filter(_.isInstanceOf[ParameterizedType])
440        .map(_.asInstanceOf[ParameterizedType])
441        .filter(e => e.getRawType.isInstanceOf[Class[_]] && e.getRawType.asInstanceOf[Class[_]].getCanonicalName.startsWith("org.apache.spark.sql.api.java.UDF"))
442      if (udfInterfaces.length == 0) {
443        throw new IOException(s"UDF class ${className} doesn't implement any UDF interface")
444      } else if (udfInterfaces.length > 1) {
445        throw new IOException(s"It is invalid to implement multiple UDF interfaces, UDF class ${className}")
446      } else {
447        try {
448          val udf = clazz.newInstance()
449          val udfReturnType = udfInterfaces(0).getActualTypeArguments.last
450          var returnType = returnDataType
451          if (returnType == null) {
452            returnType = JavaTypeInference.inferDataType(udfReturnType)._1
453          }
454
455          udfInterfaces(0).getActualTypeArguments.length match {
456            case 2 => register(name, udf.asInstanceOf[UDF1[_, _]], returnType)
457            case 3 => register(name, udf.asInstanceOf[UDF2[_, _, _]], returnType)
458            case 4 => register(name, udf.asInstanceOf[UDF3[_, _, _, _]], returnType)
459            case 5 => register(name, udf.asInstanceOf[UDF4[_, _, _, _, _]], returnType)
460            case 6 => register(name, udf.asInstanceOf[UDF5[_, _, _, _, _, _]], returnType)
461            case 7 => register(name, udf.asInstanceOf[UDF6[_, _, _, _, _, _, _]], returnType)
462            case 8 => register(name, udf.asInstanceOf[UDF7[_, _, _, _, _, _, _, _]], returnType)
463            case 9 => register(name, udf.asInstanceOf[UDF8[_, _, _, _, _, _, _, _, _]], returnType)
464            case 10 => register(name, udf.asInstanceOf[UDF9[_, _, _, _, _, _, _, _, _, _]], returnType)
465            case 11 => register(name, udf.asInstanceOf[UDF10[_, _, _, _, _, _, _, _, _, _, _]], returnType)
466            case 12 => register(name, udf.asInstanceOf[UDF11[_, _, _, _, _, _, _, _, _, _, _, _]], returnType)
467            case 13 => register(name, udf.asInstanceOf[UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _]], returnType)
468            case 14 => register(name, udf.asInstanceOf[UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType)
469            case 15 => register(name, udf.asInstanceOf[UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType)
470            case 16 => register(name, udf.asInstanceOf[UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType)
471            case 17 => register(name, udf.asInstanceOf[UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType)
472            case 18 => register(name, udf.asInstanceOf[UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType)
473            case 19 => register(name, udf.asInstanceOf[UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType)
474            case 20 => register(name, udf.asInstanceOf[UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType)
475            case 21 => register(name, udf.asInstanceOf[UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType)
476            case 22 => register(name, udf.asInstanceOf[UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType)
477            case 23 => register(name, udf.asInstanceOf[UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType)
478            case n => logError(s"UDF class with ${n} type arguments is not supported ")
479          }
480        } catch {
481          case e @ (_: InstantiationException | _: IllegalArgumentException) =>
482            logError(s"Can not instantiate class ${className}, please make sure it has public non argument constructor")
483        }
484      }
485    } catch {
486      case e: ClassNotFoundException => logError(s"Can not load class ${className}, please make sure it is on the classpath")
487    }
488
489  }
490
491  /**
492   * Register a user-defined function with 1 arguments.
493   * @since 1.3.0
494   */
495  def register(name: String, f: UDF1[_, _], returnType: DataType): Unit = {
496    val func = f.asInstanceOf[UDF1[Any, Any]].call(_: Any)
497    functionRegistry.registerFunction(
498      name,
499      (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
500  }
501
502  /**
503   * Register a user-defined function with 2 arguments.
504   * @since 1.3.0
505   */
506  def register(name: String, f: UDF2[_, _, _], returnType: DataType): Unit = {
507    val func = f.asInstanceOf[UDF2[Any, Any, Any]].call(_: Any, _: Any)
508    functionRegistry.registerFunction(
509      name,
510      (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
511  }
512
513  /**
514   * Register a user-defined function with 3 arguments.
515   * @since 1.3.0
516   */
517  def register(name: String, f: UDF3[_, _, _, _], returnType: DataType): Unit = {
518    val func = f.asInstanceOf[UDF3[Any, Any, Any, Any]].call(_: Any, _: Any, _: Any)
519    functionRegistry.registerFunction(
520      name,
521      (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
522  }
523
524  /**
525   * Register a user-defined function with 4 arguments.
526   * @since 1.3.0
527   */
528  def register(name: String, f: UDF4[_, _, _, _, _], returnType: DataType): Unit = {
529    val func = f.asInstanceOf[UDF4[Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any)
530    functionRegistry.registerFunction(
531      name,
532      (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
533  }
534
535  /**
536   * Register a user-defined function with 5 arguments.
537   * @since 1.3.0
538   */
539  def register(name: String, f: UDF5[_, _, _, _, _, _], returnType: DataType): Unit = {
540    val func = f.asInstanceOf[UDF5[Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any)
541    functionRegistry.registerFunction(
542      name,
543      (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
544  }
545
546  /**
547   * Register a user-defined function with 6 arguments.
548   * @since 1.3.0
549   */
550  def register(name: String, f: UDF6[_, _, _, _, _, _, _], returnType: DataType): Unit = {
551    val func = f.asInstanceOf[UDF6[Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any)
552    functionRegistry.registerFunction(
553      name,
554      (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
555  }
556
557  /**
558   * Register a user-defined function with 7 arguments.
559   * @since 1.3.0
560   */
561  def register(name: String, f: UDF7[_, _, _, _, _, _, _, _], returnType: DataType): Unit = {
562    val func = f.asInstanceOf[UDF7[Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any)
563    functionRegistry.registerFunction(
564      name,
565      (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
566  }
567
568  /**
569   * Register a user-defined function with 8 arguments.
570   * @since 1.3.0
571   */
572  def register(name: String, f: UDF8[_, _, _, _, _, _, _, _, _], returnType: DataType): Unit = {
573    val func = f.asInstanceOf[UDF8[Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any)
574    functionRegistry.registerFunction(
575      name,
576      (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
577  }
578
579  /**
580   * Register a user-defined function with 9 arguments.
581   * @since 1.3.0
582   */
583  def register(name: String, f: UDF9[_, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = {
584    val func = f.asInstanceOf[UDF9[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any)
585    functionRegistry.registerFunction(
586      name,
587      (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
588  }
589
590  /**
591   * Register a user-defined function with 10 arguments.
592   * @since 1.3.0
593   */
594  def register(name: String, f: UDF10[_, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = {
595    val func = 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)
596    functionRegistry.registerFunction(
597      name,
598      (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
599  }
600
601  /**
602   * Register a user-defined function with 11 arguments.
603   * @since 1.3.0
604   */
605  def register(name: String, f: UDF11[_, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = {
606    val func = 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)
607    functionRegistry.registerFunction(
608      name,
609      (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
610  }
611
612  /**
613   * Register a user-defined function with 12 arguments.
614   * @since 1.3.0
615   */
616  def register(name: String, f: UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = {
617    val func = 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)
618    functionRegistry.registerFunction(
619      name,
620      (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
621  }
622
623  /**
624   * Register a user-defined function with 13 arguments.
625   * @since 1.3.0
626   */
627  def register(name: String, f: UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = {
628    val func = 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)
629    functionRegistry.registerFunction(
630      name,
631      (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
632  }
633
634  /**
635   * Register a user-defined function with 14 arguments.
636   * @since 1.3.0
637   */
638  def register(name: String, f: UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = {
639    val func = 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)
640    functionRegistry.registerFunction(
641      name,
642      (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
643  }
644
645  /**
646   * Register a user-defined function with 15 arguments.
647   * @since 1.3.0
648   */
649  def register(name: String, f: UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = {
650    val func = 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)
651    functionRegistry.registerFunction(
652      name,
653      (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
654  }
655
656  /**
657   * Register a user-defined function with 16 arguments.
658   * @since 1.3.0
659   */
660  def register(name: String, f: UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = {
661    val func = 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)
662    functionRegistry.registerFunction(
663      name,
664      (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
665  }
666
667  /**
668   * Register a user-defined function with 17 arguments.
669   * @since 1.3.0
670   */
671  def register(name: String, f: UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = {
672    val func = 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)
673    functionRegistry.registerFunction(
674      name,
675      (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
676  }
677
678  /**
679   * Register a user-defined function with 18 arguments.
680   * @since 1.3.0
681   */
682  def register(name: String, f: UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = {
683    val func = 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)
684    functionRegistry.registerFunction(
685      name,
686      (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
687  }
688
689  /**
690   * Register a user-defined function with 19 arguments.
691   * @since 1.3.0
692   */
693  def register(name: String, f: UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = {
694    val func = 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)
695    functionRegistry.registerFunction(
696      name,
697      (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
698  }
699
700  /**
701   * Register a user-defined function with 20 arguments.
702   * @since 1.3.0
703   */
704  def register(name: String, f: UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = {
705    val func = 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)
706    functionRegistry.registerFunction(
707      name,
708      (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
709  }
710
711  /**
712   * Register a user-defined function with 21 arguments.
713   * @since 1.3.0
714   */
715  def register(name: String, f: UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = {
716    val func = 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)
717    functionRegistry.registerFunction(
718      name,
719      (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
720  }
721
722  /**
723   * Register a user-defined function with 22 arguments.
724   * @since 1.3.0
725   */
726  def register(name: String, f: UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = {
727    val func = 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)
728    functionRegistry.registerFunction(
729      name,
730      (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
731  }
732
733  // scalastyle:on line.size.limit
734
735}
736