From deb82f62b6581d79b0460a78746f4725b38e47d6 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Thu, 13 Jul 2017 15:08:47 -0700 Subject: [PATCH 01/12] fix. --- python/pyspark/sql/context.py | 27 +- .../sql/catalyst/analysis/Analyzer.scala | 2 +- .../sql/catalyst/expressions/ScalaUDF.scala | 13 +- .../apache/spark/sql/UDFRegistration.scala | 824 ++++++++++++++---- .../sql/expressions/UserDefinedFunction.scala | 56 +- .../org/apache/spark/sql/JavaRandUDF.java | 30 + .../org/apache/spark/sql/JavaUDFSuite.java | 30 + .../apache/spark/sql/SQLContextSuite.scala | 2 +- .../scala/org/apache/spark/sql/UDFSuite.scala | 22 +- 9 files changed, 820 insertions(+), 186 deletions(-) create mode 100644 sql/core/src/test/java/test/org/apache/spark/sql/JavaRandUDF.java diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index c44ab247fd3d3..a0b1ab6d7ec9b 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -28,7 +28,7 @@ from pyspark.sql.dataframe import DataFrame from pyspark.sql.readwriter import DataFrameReader from pyspark.sql.streaming import DataStreamReader -from pyspark.sql.types import IntegerType, Row, StringType +from pyspark.sql.types import DoubleType, IntegerType, Row, StringType from pyspark.sql.utils import install_exception_handler __all__ = ["SQLContext", "HiveContext", "UDFRegistration"] @@ -208,29 +208,40 @@ def registerFunction(self, name, f, returnType=StringType()): @ignore_unicode_prefix @since(2.1) - def registerJavaFunction(self, name, javaClassName, returnType=None): + def registerJavaFunction(self, name, javaClassName, returnType=None, deterministic=True, + distinctLike=False): """Register a java UDF so it can be used in SQL statements. In addition to a name and the function itself, the return type can be optionally specified. When the return type is not specified we would infer it via reflection. - :param name: name of the UDF - :param javaClassName: fully qualified name of java class - :param returnType: a :class:`pyspark.sql.types.DataType` object + + :param name: name of the UDF. + :param javaClassName: fully qualified name of java class. + :param returnType: a :class:`pyspark.sql.types.DataType` object. + :param deterministic: a flag indicating if the UDF is deterministic. Deterministic UDF + returns same result each time it is invoked with a particular input. + :param distinctLike: a UDF is considered distinctLike if the UDF can be evaluated on just + the distinct values of a column. >>> sqlContext.registerJavaFunction("javaStringLength", ... "test.org.apache.spark.sql.JavaStringLength", IntegerType()) >>> sqlContext.sql("SELECT javaStringLength('test')").collect() - [Row(UDF(test)=4)] + [Row(UDF:javaStringLength(test)=4)] >>> sqlContext.registerJavaFunction("javaStringLength2", ... "test.org.apache.spark.sql.JavaStringLength") >>> sqlContext.sql("SELECT javaStringLength2('test')").collect() - [Row(UDF(test)=4)] + [Row(UDF:javaStringLength2(test)=4)] + >>> sqlContext.registerJavaFunction("javaRand", + ... "test.org.apache.spark.sql.JavaRandUDF", DoubleType(), deterministic=False) + >>> sqlContext.sql("SELECT javaRand(3)").collect() # doctest: +SKIP + [Row(UDF:javaRand(3)=3.12345)] """ jdt = None if returnType is not None: jdt = self.sparkSession._jsparkSession.parseDataType(returnType.json()) - self.sparkSession._jsparkSession.udf().registerJava(name, javaClassName, jdt) + self.sparkSession._jsparkSession.udf().registerJava( + name, javaClassName, jdt, deterministic, distinctLike) @ignore_unicode_prefix @since(2.3) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 7745709e07fe5..c0e1c2e208033 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1950,7 +1950,7 @@ class Analyzer( case p => p transformExpressionsUp { - case udf @ ScalaUDF(func, _, inputs, _, _, _) => + case udf @ ScalaUDF(func, _, inputs, _, _, _, _, _) => val parameterTypes = ScalaReflection.getParameterTypes(func) assert(parameterTypes.length == inputs.length) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala index a54f6d0e11147..ced066b052fc2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala @@ -24,7 +24,6 @@ import org.apache.spark.sql.types.DataType /** * User-defined function. - * Note that the user-defined functions must be deterministic. * @param function The user defined scala function to run. * Note that if you use primitive parameters, you are not able to check if it is * null or not, and the UDF will return null for you if the primitive input is @@ -35,8 +34,12 @@ import org.apache.spark.sql.types.DataType * not want to perform coercion, simply use "Nil". Note that it would've been * better to use Option of Seq[DataType] so we can use "None" as the case for no * type coercion. However, that would require more refactoring of the codebase. - * @param udfName The user-specified name of this UDF. + * @param udfName The user-specified name of this UDF. * @param nullable True if the UDF can return null value. + * @param udfDeterministic True if the UDF is deterministic. Deterministic UDF returns same result + * each time it is invoked with a particular input. + * @param distinctLike A UDF is considered distinctLike if the UDF can be evaluated on just the + * distinct values of a column. */ case class ScalaUDF( function: AnyRef, @@ -44,9 +47,13 @@ case class ScalaUDF( children: Seq[Expression], inputTypes: Seq[DataType] = Nil, udfName: Option[String] = None, - nullable: Boolean = true) + nullable: Boolean = true, + udfDeterministic: Boolean = true, + distinctLike: Boolean = false) extends Expression with ImplicitCastInputTypes with NonSQLExpression { + override def deterministic: Boolean = udfDeterministic && children.forall(_.deterministic) + override def toString: String = s"${udfName.map(name => s"UDF:$name").getOrElse("UDF")}(${children.mkString(", ")})" 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 c66d4057b9135..e331f8faa2358 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 @@ -104,21 +104,33 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends val inputTypes = (1 to x).foldRight("Nil")((i, s) => {s"ScalaReflection.schemaFor[A$i].dataType :: $s"}) println(s""" /** - * Register a Scala closure of ${x} arguments as user-defined function (UDF). + * Registers a Scala closure of ${x} arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ def register[$typeTags](name: String, func: Function$x[$types]): UserDefinedFunction = { + register(name, func, deterministic = true, distinctLike = false) + } + + /** + * Registers a Scala closure of ${x} arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @since 2.3.0 + */ + def register[$typeTags](name: String, func: Function$x[$types], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputTypes = Try($inputTypes).toOption def builder(e: Seq[Expression]) = if (e.length == $x) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: $x; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf + val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism + withDistinctLike }""") } @@ -130,442 +142,726 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends val version = if (i == 0) "2.3.0" else "1.3.0" val funcCall = if (i == 0) "() => func" else "func" println(s""" - |/** - | * Register a user-defined function with ${i} arguments. - | * @since $version - | */ - |def register(name: String, f: UDF$i[$extTypeArgs], returnType: DataType): Unit = { - | val func = f$anyCast.call($anyParams) - | def builder(e: Seq[Expression]) = if (e.length == $i) { - | ScalaUDF($funcCall, returnType, e) - | } else { - | throw new AnalysisException("Invalid number of arguments for function " + name + - | ". Expected: $i; Found: " + e.length) - | } - | functionRegistry.createOrReplaceTempFunction(name, builder) - |}""".stripMargin) + |/** + | * Registers a user-defined function with ${i} arguments. + | * @since $version + | */ + |def register(name: String, f: UDF$i[$extTypeArgs], returnType: DataType): Unit = { + | register(name, f, returnType, deterministic = true, distinctLike = false) + |} + | + |/** + | * Registers a user-defined function with ${i} arguments. + | * @since 2.3.0 + | */ + |def register(name: String, f: UDF$i[$extTypeArgs], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { + | val func = f$anyCast.call($anyParams) + | def builder(e: Seq[Expression]) = if (e.length == $i) { + | ScalaUDF($funcCall, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) + | } else { + | throw new AnalysisException("Invalid number of arguments for function " + name + + | ". Expected: $i; Found: " + e.length) + | } + | functionRegistry.createOrReplaceTempFunction(name, builder) + |}""".stripMargin) } */ /** - * Register a Scala closure of 0 arguments as user-defined function (UDF). + * Registers a Scala closure of 0 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ def register[RT: TypeTag](name: String, func: Function0[RT]): UserDefinedFunction = { + register(name, func, deterministic = true, distinctLike = false) + } + + /** + * Registers a Scala closure of 0 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @since 2.3.0 + */ + def register[RT: TypeTag](name: String, func: Function0[RT], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputTypes = Try(Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 0) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 0; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf + val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism + withDistinctLike } /** - * Register a Scala closure of 1 arguments as user-defined function (UDF). + * Registers a Scala closure of 1 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag](name: String, func: Function1[A1, RT]): UserDefinedFunction = { + register(name, func, deterministic = true, distinctLike = false) + } + + /** + * Registers a Scala closure of 1 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @since 2.3.0 + */ + def register[RT: TypeTag, A1: TypeTag](name: String, func: Function1[A1, RT], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 1) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 1; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf + val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism + withDistinctLike } /** - * Register a Scala closure of 2 arguments as user-defined function (UDF). + * Registers a Scala closure of 2 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag](name: String, func: Function2[A1, A2, RT]): UserDefinedFunction = { + register(name, func, deterministic = true, distinctLike = false) + } + + /** + * Registers a Scala closure of 2 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @since 2.3.0 + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag](name: String, func: Function2[A1, A2, RT], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 2) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 2; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf + val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism + withDistinctLike } /** - * Register a Scala closure of 3 arguments as user-defined function (UDF). + * Registers a Scala closure of 3 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag](name: String, func: Function3[A1, A2, A3, RT]): UserDefinedFunction = { + register(name, func, deterministic = true, distinctLike = false) + } + + /** + * Registers a Scala closure of 3 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @since 2.3.0 + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag](name: String, func: Function3[A1, A2, A3, RT], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 3) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 3; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf + val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism + withDistinctLike } /** - * Register a Scala closure of 4 arguments as user-defined function (UDF). + * Registers a Scala closure of 4 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag](name: String, func: Function4[A1, A2, A3, A4, RT]): UserDefinedFunction = { + register(name, func, deterministic = true, distinctLike = false) + } + + /** + * Registers a Scala closure of 4 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @since 2.3.0 + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag](name: String, func: Function4[A1, A2, A3, A4, RT], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 4) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 4; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf + val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism + withDistinctLike } /** - * Register a Scala closure of 5 arguments as user-defined function (UDF). + * Registers a Scala closure of 5 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ 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 = { + register(name, func, deterministic = true, distinctLike = false) + } + + /** + * Registers a Scala closure of 5 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @since 2.3.0 + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag](name: String, func: Function5[A1, A2, A3, A4, A5, RT], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 5) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 5; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf + val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism + withDistinctLike } /** - * Register a Scala closure of 6 arguments as user-defined function (UDF). + * Registers a Scala closure of 6 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ 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 = { + register(name, func, deterministic = true, distinctLike = false) + } + + /** + * Registers a Scala closure of 6 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @since 2.3.0 + */ + 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], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 6) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 6; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf + val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism + withDistinctLike } /** - * Register a Scala closure of 7 arguments as user-defined function (UDF). + * Registers a Scala closure of 7 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ 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 = { + register(name, func, deterministic = true, distinctLike = false) + } + + /** + * Registers a Scala closure of 7 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @since 2.3.0 + */ + 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], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 7) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 7; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf + val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism + withDistinctLike } /** - * Register a Scala closure of 8 arguments as user-defined function (UDF). + * Registers a Scala closure of 8 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ 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 = { + register(name, func, deterministic = true, distinctLike = false) + } + + /** + * Registers a Scala closure of 8 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @since 2.3.0 + */ + 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], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 8) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 8; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf + val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism + withDistinctLike } /** - * Register a Scala closure of 9 arguments as user-defined function (UDF). + * Registers a Scala closure of 9 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ 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 = { + register(name, func, deterministic = true, distinctLike = false) + } + + /** + * Registers a Scala closure of 9 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @since 2.3.0 + */ + 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], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 9) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 9; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf + val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism + withDistinctLike } /** - * Register a Scala closure of 10 arguments as user-defined function (UDF). + * Registers a Scala closure of 10 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ 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 = { + register(name, func, deterministic = true, distinctLike = false) + } + + /** + * Registers a Scala closure of 10 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @since 2.3.0 + */ + 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], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 10) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 10; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf + val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism + withDistinctLike } /** - * Register a Scala closure of 11 arguments as user-defined function (UDF). + * Registers a Scala closure of 11 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ 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 = { + register(name, func, deterministic = true, distinctLike = false) + } + + /** + * Registers a Scala closure of 11 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @since 2.3.0 + */ + 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], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 11) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 11; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf + val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism + withDistinctLike } /** - * Register a Scala closure of 12 arguments as user-defined function (UDF). + * Registers a Scala closure of 12 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ 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 = { + register(name, func, deterministic = true, distinctLike = false) + } + + /** + * Registers a Scala closure of 12 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @since 2.3.0 + */ + 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], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 12) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 12; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf + val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism + withDistinctLike } /** - * Register a Scala closure of 13 arguments as user-defined function (UDF). + * Registers a Scala closure of 13 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ 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 = { + register(name, func, deterministic = true, distinctLike = false) + } + + /** + * Registers a Scala closure of 13 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @since 2.3.0 + */ + 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], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 13) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 13; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf + val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism + withDistinctLike } /** - * Register a Scala closure of 14 arguments as user-defined function (UDF). + * Registers a Scala closure of 14 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ 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 = { + register(name, func, deterministic = true, distinctLike = false) + } + + /** + * Registers a Scala closure of 14 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @since 2.3.0 + */ + 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], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 14) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 14; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf + val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism + withDistinctLike } /** - * Register a Scala closure of 15 arguments as user-defined function (UDF). + * Registers a Scala closure of 15 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ 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 = { + register(name, func, deterministic = true, distinctLike = false) + } + + /** + * Registers a Scala closure of 15 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @since 2.3.0 + */ + 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], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 15) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 15; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf + val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism + withDistinctLike } /** - * Register a Scala closure of 16 arguments as user-defined function (UDF). + * Registers a Scala closure of 16 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ 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 = { + register(name, func, deterministic = true, distinctLike = false) + } + + /** + * Registers a Scala closure of 16 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @since 2.3.0 + */ + 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], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 16) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 16; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf + val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism + withDistinctLike } /** - * Register a Scala closure of 17 arguments as user-defined function (UDF). + * Registers a Scala closure of 17 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ 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 = { + register(name, func, deterministic = true, distinctLike = false) + } + + /** + * Registers a Scala closure of 17 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @since 2.3.0 + */ + 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], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 17) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 17; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf + val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism + withDistinctLike } /** - * Register a Scala closure of 18 arguments as user-defined function (UDF). + * Registers a Scala closure of 18 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ 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 = { + register(name, func, deterministic = true, distinctLike = false) + } + + /** + * Registers a Scala closure of 18 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @since 2.3.0 + */ + 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], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 18) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 18; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf + val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism + withDistinctLike } /** - * Register a Scala closure of 19 arguments as user-defined function (UDF). + * Registers a Scala closure of 19 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ 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 = { + register(name, func, deterministic = true, distinctLike = false) + } + + /** + * Registers a Scala closure of 19 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @since 2.3.0 + */ + 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], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 19) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 19; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf + val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism + withDistinctLike } /** - * Register a Scala closure of 20 arguments as user-defined function (UDF). + * Registers a Scala closure of 20 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ 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 = { + register(name, func, deterministic = true, distinctLike = false) + } + + /** + * Registers a Scala closure of 20 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @since 2.3.0 + */ + 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], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 20) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 20; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf + val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism + withDistinctLike } /** - * Register a Scala closure of 21 arguments as user-defined function (UDF). + * Registers a Scala closure of 21 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ 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 = { + register(name, func, deterministic = true, distinctLike = false) + } + + /** + * Registers a Scala closure of 21 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @since 2.3.0 + */ + 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], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 21) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 21; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf + val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism + withDistinctLike } /** - * Register a Scala closure of 22 arguments as user-defined function (UDF). + * Registers a Scala closure of 22 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ 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 = { + register(name, func, deterministic = true, distinctLike = false) + } + + /** + * Registers a Scala closure of 22 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @since 2.3.0 + */ + 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], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 22) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 22; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) + val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf + val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism + withDistinctLike } ////////////////////////////////////////////////////////////////////////////////////////////// ////////////////////////////////////////////////////////////////////////////////////////////// /** - * Register a Java UDF class using reflection, for use from pyspark + * Registers a Java UDF class using reflection, for use from pyspark * * @param name udf name * @param className fully qualified class name of udf @@ -573,7 +869,26 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * via reflection. */ private[sql] def registerJava(name: String, className: String, returnDataType: DataType): Unit = { - + registerJava(name, className, returnDataType, deterministic = true, distinctLike = false) + } + /** + * Registers a Java UDF class using reflection, for use from pyspark + * + * @param name udf name + * @param className fully qualified class name of udf + * @param returnDataType return type of udf. If it is null, spark would try to infer + * via reflection. + * @param deterministic True if the UDF is deterministic. Deterministic UDF returns same result + * each time it is invoked with a particular input. + * @param distinctLike A UDF is considered distinctLike if the UDF can be evaluated on just the + * distinct values of a column. + */ + private[sql] def registerJava( + name: String, + className: String, + returnDataType: DataType, + deterministic: Boolean, + distinctLike: Boolean): Unit = { try { val clazz = Utils.classForName(className) val udfInterfaces = clazz.getGenericInterfaces @@ -581,9 +896,9 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends .map(_.asInstanceOf[ParameterizedType]) .filter(e => e.getRawType.isInstanceOf[Class[_]] && e.getRawType.asInstanceOf[Class[_]].getCanonicalName.startsWith("org.apache.spark.sql.api.java.UDF")) if (udfInterfaces.length == 0) { - throw new AnalysisException(s"UDF class ${className} doesn't implement any UDF interface") + throw new AnalysisException(s"UDF class $className doesn't implement any UDF interface") } else if (udfInterfaces.length > 1) { - throw new AnalysisException(s"It is invalid to implement multiple UDF interfaces, UDF class ${className}") + throw new AnalysisException(s"It is invalid to implement multiple UDF interfaces, UDF class $className") } else { try { val udf = clazz.newInstance() @@ -594,39 +909,39 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } udfInterfaces(0).getActualTypeArguments.length match { - case 1 => register(name, udf.asInstanceOf[UDF0[_]], returnType) - case 2 => register(name, udf.asInstanceOf[UDF1[_, _]], returnType) - case 3 => register(name, udf.asInstanceOf[UDF2[_, _, _]], returnType) - case 4 => register(name, udf.asInstanceOf[UDF3[_, _, _, _]], returnType) - case 5 => register(name, udf.asInstanceOf[UDF4[_, _, _, _, _]], returnType) - case 6 => register(name, udf.asInstanceOf[UDF5[_, _, _, _, _, _]], returnType) - case 7 => register(name, udf.asInstanceOf[UDF6[_, _, _, _, _, _, _]], returnType) - case 8 => register(name, udf.asInstanceOf[UDF7[_, _, _, _, _, _, _, _]], returnType) - case 9 => register(name, udf.asInstanceOf[UDF8[_, _, _, _, _, _, _, _, _]], returnType) - case 10 => register(name, udf.asInstanceOf[UDF9[_, _, _, _, _, _, _, _, _, _]], returnType) - case 11 => register(name, udf.asInstanceOf[UDF10[_, _, _, _, _, _, _, _, _, _, _]], returnType) - case 12 => register(name, udf.asInstanceOf[UDF11[_, _, _, _, _, _, _, _, _, _, _, _]], returnType) - case 13 => register(name, udf.asInstanceOf[UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) - case 14 => register(name, udf.asInstanceOf[UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) - case 15 => register(name, udf.asInstanceOf[UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) - case 16 => register(name, udf.asInstanceOf[UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) - case 17 => register(name, udf.asInstanceOf[UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) - case 18 => register(name, udf.asInstanceOf[UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) - case 19 => register(name, udf.asInstanceOf[UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) - case 20 => register(name, udf.asInstanceOf[UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) - case 21 => register(name, udf.asInstanceOf[UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) - case 22 => register(name, udf.asInstanceOf[UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) - case 23 => register(name, udf.asInstanceOf[UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) + case 1 => register(name, udf.asInstanceOf[UDF0[_]], returnType, deterministic, distinctLike) + case 2 => register(name, udf.asInstanceOf[UDF1[_, _]], returnType, deterministic, distinctLike) + case 3 => register(name, udf.asInstanceOf[UDF2[_, _, _]], returnType, deterministic, distinctLike) + case 4 => register(name, udf.asInstanceOf[UDF3[_, _, _, _]], returnType, deterministic, distinctLike) + case 5 => register(name, udf.asInstanceOf[UDF4[_, _, _, _, _]], returnType, deterministic, distinctLike) + case 6 => register(name, udf.asInstanceOf[UDF5[_, _, _, _, _, _]], returnType, deterministic, distinctLike) + case 7 => register(name, udf.asInstanceOf[UDF6[_, _, _, _, _, _, _]], returnType, deterministic, distinctLike) + case 8 => register(name, udf.asInstanceOf[UDF7[_, _, _, _, _, _, _, _]], returnType, deterministic, distinctLike) + case 9 => register(name, udf.asInstanceOf[UDF8[_, _, _, _, _, _, _, _, _]], returnType, deterministic, distinctLike) + case 10 => register(name, udf.asInstanceOf[UDF9[_, _, _, _, _, _, _, _, _, _]], returnType, deterministic, distinctLike) + case 11 => register(name, udf.asInstanceOf[UDF10[_, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic, distinctLike) + case 12 => register(name, udf.asInstanceOf[UDF11[_, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic, distinctLike) + case 13 => register(name, udf.asInstanceOf[UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic, distinctLike) + case 14 => register(name, udf.asInstanceOf[UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic, distinctLike) + case 15 => register(name, udf.asInstanceOf[UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic, distinctLike) + case 16 => register(name, udf.asInstanceOf[UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic, distinctLike) + case 17 => register(name, udf.asInstanceOf[UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic, distinctLike) + case 18 => register(name, udf.asInstanceOf[UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic, distinctLike) + case 19 => register(name, udf.asInstanceOf[UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic, distinctLike) + case 20 => register(name, udf.asInstanceOf[UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic, distinctLike) + case 21 => register(name, udf.asInstanceOf[UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic, distinctLike) + case 22 => register(name, udf.asInstanceOf[UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic, distinctLike) + case 23 => register(name, udf.asInstanceOf[UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic, distinctLike) case n => - throw new AnalysisException(s"UDF class with ${n} type arguments is not supported.") + throw new AnalysisException(s"UDF class with $n type arguments is not supported.") } } catch { case e @ (_: InstantiationException | _: IllegalArgumentException) => - throw new AnalysisException(s"Can not instantiate class ${className}, please make sure it has public non argument constructor") + throw new AnalysisException(s"Can not instantiate class $className, please make sure it has public non argument constructor") } } } catch { - case e: ClassNotFoundException => throw new AnalysisException(s"Can not load class ${className}, please make sure it is on the classpath") + case e: ClassNotFoundException => throw new AnalysisException(s"Can not load class $className, please make sure it is on the classpath") } } @@ -652,14 +967,23 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } } + /** - * Register a user-defined function with 0 arguments. + * Registers a user-defined function with 0 arguments. * @since 2.3.0 */ def register(name: String, f: UDF0[_], returnType: DataType): Unit = { + register(name, f, returnType, deterministic = true, distinctLike = false) + } + + /** + * Registers a user-defined function with 0 arguments. + * @since 2.3.0 + */ + def register(name: String, f: UDF0[_], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { val func = f.asInstanceOf[UDF0[Any]].call() def builder(e: Seq[Expression]) = if (e.length == 0) { - ScalaUDF(() => func, returnType, e) + ScalaUDF(() => func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 0; Found: " + e.length) @@ -668,13 +992,21 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Register a user-defined function with 1 arguments. + * Registers a user-defined function with 1 arguments. * @since 1.3.0 */ def register(name: String, f: UDF1[_, _], returnType: DataType): Unit = { + register(name, f, returnType, deterministic = true, distinctLike = false) + } + + /** + * Registers a user-defined function with 1 arguments. + * @since 2.3.0 + */ + def register(name: String, f: UDF1[_, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { val func = f.asInstanceOf[UDF1[Any, Any]].call(_: Any) def builder(e: Seq[Expression]) = if (e.length == 1) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 1; Found: " + e.length) @@ -683,13 +1015,21 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Register a user-defined function with 2 arguments. + * Registers a user-defined function with 2 arguments. * @since 1.3.0 */ def register(name: String, f: UDF2[_, _, _], returnType: DataType): Unit = { + register(name, f, returnType, deterministic = true, distinctLike = false) + } + + /** + * Registers a user-defined function with 2 arguments. + * @since 2.3.0 + */ + def register(name: String, f: UDF2[_, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { val func = f.asInstanceOf[UDF2[Any, Any, Any]].call(_: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 2) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 2; Found: " + e.length) @@ -698,13 +1038,21 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Register a user-defined function with 3 arguments. + * Registers a user-defined function with 3 arguments. * @since 1.3.0 */ def register(name: String, f: UDF3[_, _, _, _], returnType: DataType): Unit = { + register(name, f, returnType, deterministic = true, distinctLike = false) + } + + /** + * Registers a user-defined function with 3 arguments. + * @since 2.3.0 + */ + def register(name: String, f: UDF3[_, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { val func = f.asInstanceOf[UDF3[Any, Any, Any, Any]].call(_: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 3) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 3; Found: " + e.length) @@ -713,13 +1061,21 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Register a user-defined function with 4 arguments. + * Registers a user-defined function with 4 arguments. * @since 1.3.0 */ def register(name: String, f: UDF4[_, _, _, _, _], returnType: DataType): Unit = { + register(name, f, returnType, deterministic = true, distinctLike = false) + } + + /** + * Registers a user-defined function with 4 arguments. + * @since 2.3.0 + */ + def register(name: String, f: UDF4[_, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { val func = f.asInstanceOf[UDF4[Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 4) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 4; Found: " + e.length) @@ -728,13 +1084,21 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Register a user-defined function with 5 arguments. + * Registers a user-defined function with 5 arguments. * @since 1.3.0 */ def register(name: String, f: UDF5[_, _, _, _, _, _], returnType: DataType): Unit = { + register(name, f, returnType, deterministic = true, distinctLike = false) + } + + /** + * Registers a user-defined function with 5 arguments. + * @since 2.3.0 + */ + def register(name: String, f: UDF5[_, _, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { val func = f.asInstanceOf[UDF5[Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 5) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 5; Found: " + e.length) @@ -743,13 +1107,21 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Register a user-defined function with 6 arguments. + * Registers a user-defined function with 6 arguments. * @since 1.3.0 */ def register(name: String, f: UDF6[_, _, _, _, _, _, _], returnType: DataType): Unit = { + register(name, f, returnType, deterministic = true, distinctLike = false) + } + + /** + * Registers a user-defined function with 6 arguments. + * @since 2.3.0 + */ + def register(name: String, f: UDF6[_, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { val func = f.asInstanceOf[UDF6[Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 6) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 6; Found: " + e.length) @@ -758,13 +1130,21 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Register a user-defined function with 7 arguments. + * Registers a user-defined function with 7 arguments. * @since 1.3.0 */ def register(name: String, f: UDF7[_, _, _, _, _, _, _, _], returnType: DataType): Unit = { + register(name, f, returnType, deterministic = true, distinctLike = false) + } + + /** + * Registers a user-defined function with 7 arguments. + * @since 2.3.0 + */ + def register(name: String, f: UDF7[_, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { val func = f.asInstanceOf[UDF7[Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 7) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 7; Found: " + e.length) @@ -773,13 +1153,21 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Register a user-defined function with 8 arguments. + * Registers a user-defined function with 8 arguments. * @since 1.3.0 */ def register(name: String, f: UDF8[_, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + register(name, f, returnType, deterministic = true, distinctLike = false) + } + + /** + * Registers a user-defined function with 8 arguments. + * @since 2.3.0 + */ + def register(name: String, f: UDF8[_, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { val func = f.asInstanceOf[UDF8[Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 8) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 8; Found: " + e.length) @@ -788,13 +1176,21 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Register a user-defined function with 9 arguments. + * Registers a user-defined function with 9 arguments. * @since 1.3.0 */ def register(name: String, f: UDF9[_, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + register(name, f, returnType, deterministic = true, distinctLike = false) + } + + /** + * Registers a user-defined function with 9 arguments. + * @since 2.3.0 + */ + def register(name: String, f: UDF9[_, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 9) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 9; Found: " + e.length) @@ -803,13 +1199,21 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Register a user-defined function with 10 arguments. + * Registers a user-defined function with 10 arguments. * @since 1.3.0 */ def register(name: String, f: UDF10[_, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + register(name, f, returnType, deterministic = true, distinctLike = false) + } + + /** + * Registers a user-defined function with 10 arguments. + * @since 2.3.0 + */ + def register(name: String, f: UDF10[_, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 10) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 10; Found: " + e.length) @@ -818,13 +1222,21 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Register a user-defined function with 11 arguments. + * Registers a user-defined function with 11 arguments. * @since 1.3.0 */ def register(name: String, f: UDF11[_, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + register(name, f, returnType, deterministic = true, distinctLike = false) + } + + /** + * Registers a user-defined function with 11 arguments. + * @since 2.3.0 + */ + def register(name: String, f: UDF11[_, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 11) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 11; Found: " + e.length) @@ -833,13 +1245,21 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Register a user-defined function with 12 arguments. + * Registers a user-defined function with 12 arguments. * @since 1.3.0 */ def register(name: String, f: UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + register(name, f, returnType, deterministic = true, distinctLike = false) + } + + /** + * Registers a user-defined function with 12 arguments. + * @since 2.3.0 + */ + def register(name: String, f: UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 12) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 12; Found: " + e.length) @@ -848,13 +1268,21 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Register a user-defined function with 13 arguments. + * Registers a user-defined function with 13 arguments. * @since 1.3.0 */ def register(name: String, f: UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + register(name, f, returnType, deterministic = true, distinctLike = false) + } + + /** + * Registers a user-defined function with 13 arguments. + * @since 2.3.0 + */ + def register(name: String, f: UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 13) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 13; Found: " + e.length) @@ -863,13 +1291,21 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Register a user-defined function with 14 arguments. + * Registers a user-defined function with 14 arguments. * @since 1.3.0 */ def register(name: String, f: UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + register(name, f, returnType, deterministic = true, distinctLike = false) + } + + /** + * Registers a user-defined function with 14 arguments. + * @since 2.3.0 + */ + def register(name: String, f: UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 14) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 14; Found: " + e.length) @@ -878,13 +1314,21 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Register a user-defined function with 15 arguments. + * Registers a user-defined function with 15 arguments. * @since 1.3.0 */ def register(name: String, f: UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + register(name, f, returnType, deterministic = true, distinctLike = false) + } + + /** + * Registers a user-defined function with 15 arguments. + * @since 2.3.0 + */ + def register(name: String, f: UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 15) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 15; Found: " + e.length) @@ -893,13 +1337,21 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Register a user-defined function with 16 arguments. + * Registers a user-defined function with 16 arguments. * @since 1.3.0 */ def register(name: String, f: UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + register(name, f, returnType, deterministic = true, distinctLike = false) + } + + /** + * Registers a user-defined function with 16 arguments. + * @since 2.3.0 + */ + def register(name: String, f: UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 16) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 16; Found: " + e.length) @@ -908,13 +1360,21 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Register a user-defined function with 17 arguments. + * Registers a user-defined function with 17 arguments. * @since 1.3.0 */ def register(name: String, f: UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + register(name, f, returnType, deterministic = true, distinctLike = false) + } + + /** + * Registers a user-defined function with 17 arguments. + * @since 2.3.0 + */ + def register(name: String, f: UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 17) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 17; Found: " + e.length) @@ -923,13 +1383,21 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Register a user-defined function with 18 arguments. + * Registers a user-defined function with 18 arguments. * @since 1.3.0 */ def register(name: String, f: UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + register(name, f, returnType, deterministic = true, distinctLike = false) + } + + /** + * Registers a user-defined function with 18 arguments. + * @since 2.3.0 + */ + def register(name: String, f: UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 18) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 18; Found: " + e.length) @@ -938,13 +1406,21 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Register a user-defined function with 19 arguments. + * Registers a user-defined function with 19 arguments. * @since 1.3.0 */ def register(name: String, f: UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + register(name, f, returnType, deterministic = true, distinctLike = false) + } + + /** + * Registers a user-defined function with 19 arguments. + * @since 2.3.0 + */ + def register(name: String, f: UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 19) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 19; Found: " + e.length) @@ -953,13 +1429,21 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Register a user-defined function with 20 arguments. + * Registers a user-defined function with 20 arguments. * @since 1.3.0 */ def register(name: String, f: UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + register(name, f, returnType, deterministic = true, distinctLike = false) + } + + /** + * Registers a user-defined function with 20 arguments. + * @since 2.3.0 + */ + def register(name: String, f: UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 20) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 20; Found: " + e.length) @@ -968,13 +1452,21 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Register a user-defined function with 21 arguments. + * Registers a user-defined function with 21 arguments. * @since 1.3.0 */ def register(name: String, f: UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + register(name, f, returnType, deterministic = true, distinctLike = false) + } + + /** + * Registers a user-defined function with 21 arguments. + * @since 2.3.0 + */ + def register(name: String, f: UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 21) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 21; Found: " + e.length) @@ -983,13 +1475,21 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Register a user-defined function with 22 arguments. + * Registers a user-defined function with 22 arguments. * @since 1.3.0 */ def register(name: String, f: UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + register(name, f, returnType, deterministic = true, distinctLike = false) + } + + /** + * Registers a user-defined function with 22 arguments. + * @since 2.3.0 + */ + def register(name: String, f: UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 22) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 22; Found: " + e.length) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala index 0c5f1b436591d..a6aada020cf1c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala @@ -35,10 +35,6 @@ import org.apache.spark.sql.types.DataType * df.select( predict(df("score")) ) * }}} * - * @note The user-defined functions must be deterministic. Due to optimization, - * duplicate invocations may be eliminated or the function may even be invoked more times than - * it is present in the query. - * * @since 1.3.0 */ @InterfaceStability.Stable @@ -49,6 +45,8 @@ case class UserDefinedFunction protected[sql] ( private var _nameOption: Option[String] = None private var _nullable: Boolean = true + private var _deterministic: Boolean = true + private var _distinctLike: Boolean = false /** * Returns true when the UDF can return a nullable value. @@ -57,6 +55,13 @@ case class UserDefinedFunction protected[sql] ( */ def nullable: Boolean = _nullable + /** + * Returns true when the UDF is deterministic. + * + * @since 2.3.0 + */ + def deterministic: Boolean = _deterministic + /** * Returns an expression that invokes the UDF, using the given arguments. * @@ -69,13 +74,17 @@ case class UserDefinedFunction protected[sql] ( exprs.map(_.expr), inputTypes.getOrElse(Nil), udfName = _nameOption, - nullable = _nullable)) + nullable = _nullable, + udfDeterministic = _deterministic, + distinctLike = _distinctLike)) } private def copyAll(): UserDefinedFunction = { val udf = copy() udf._nameOption = _nameOption udf._nullable = _nullable + udf._deterministic = _deterministic + udf._distinctLike = _distinctLike udf } @@ -84,9 +93,10 @@ case class UserDefinedFunction protected[sql] ( * * @since 2.3.0 */ - def withName(name: String): this.type = { - this._nameOption = Option(name) - this + def withName(name: String): UserDefinedFunction = { + val udf = copyAll() + udf._nameOption = Option(name) + udf } /** @@ -103,4 +113,34 @@ case class UserDefinedFunction protected[sql] ( udf } } + + /** + * Updates UserDefinedFunction to non-deterministic. + * + * @since 2.3.0 + */ + def nonDeterministic(): UserDefinedFunction = { + if (!_deterministic) { + this + } else { + val udf = copyAll() + udf._deterministic = false + udf + } + } + + /** + * Updates UserDefinedFunction to distinctLike. + * + * @since 2.3.0 + */ + def withDistinctLike(): UserDefinedFunction = { + if (_distinctLike) { + this + } else { + val udf = copyAll() + udf._distinctLike = true + udf + } + } } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaRandUDF.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaRandUDF.java new file mode 100644 index 0000000000000..5bf4d9f19700a --- /dev/null +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaRandUDF.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package test.org.apache.spark.sql; + +import org.apache.spark.sql.api.java.UDF1; + +/** + * It is used for register Java UDF from PySpark + */ +public class JavaRandUDF implements UDF1 { + @Override + public Double call(Integer i) { + return i + Math.random(); + } +} \ No newline at end of file diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java index 5bf1888826186..d46c4a7dc0ae9 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java @@ -28,6 +28,7 @@ import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.api.java.UDF1; import org.apache.spark.sql.api.java.UDF2; import org.apache.spark.sql.types.DataTypes; @@ -121,4 +122,33 @@ public void udf6Test() { Row result = spark.sql("SELECT returnOne()").head(); Assert.assertEquals(1, result.getInt(0)); } + + public static class randUDFTest implements UDF1 { + @Override + public Double call(Integer i) { + return i + Math.random(); + } + } + + @SuppressWarnings("unchecked") + @Test + public void udf7Test() { + spark.udf().registerJava( + "randUDF", randUDFTest.class.getName(), DataTypes.DoubleType, false, false); + + Row result = spark.sql("SELECT randUDF(1)").head(); + Assert.assertTrue(result.getDouble(0) >= 0.0); + } + + @SuppressWarnings("unchecked") + @Test + public void udf8Test() { + spark.udf().register( + "randUDF", (Integer i) -> i + Math.random(), DataTypes.DoubleType, false, false); + + Row result = spark.sql("SELECT randUDF(1)").head(); + Assert.assertTrue(result.getDouble(0) >= 0.0); + Assert.assertTrue("EXPLAIN outputs are expected to contain the UDF name.", + spark.sql("EXPLAIN SELECT randUDF(1) AS r").collectAsList().toString().contains("randUDF")); + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala index 2b35db411e2ab..7b91afd9d7577 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala @@ -69,7 +69,7 @@ class SQLContextSuite extends SparkFunSuite with SharedSparkContext { // UDF should not be shared def myadd(a: Int, b: Int): Int = a + b - session1.udf.register[Int, Int, Int]("myadd", myadd) + session1.udf.register[Int, Int, Int]("myadd", myadd _) session1.sql("select myadd(1, 2)").explain() intercept[AnalysisException] { session2.sql("select myadd(1, 2)").explain() 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 335b882ace92a..d31d6937980be 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 @@ -17,9 +17,12 @@ package org.apache.spark.sql +import org.apache.spark.sql.catalyst.plans.logical.Project import org.apache.spark.sql.execution.command.ExplainCommand +import org.apache.spark.sql.functions.{col, udf} import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.test.SQLTestData._ +import org.apache.spark.sql.types.DataTypes private case class FunctionResult(f1: String, f2: String) @@ -109,9 +112,22 @@ class UDFSuite extends QueryTest with SharedSQLContext { assert(sql("select foo(5)").head().getInt(0) == 6) } - test("ZeroArgument UDF") { - spark.udf.register("random0", () => { Math.random()}) - assert(sql("SELECT random0()").head().getDouble(0) >= 0.0) + test("ZeroArgument non-deterministic UDF") { + spark.udf.register( + "random0", () => { Math.random() }, deterministic = false, distinctLike = false) + val df = sql("SELECT random0()") + assert(df.logicalPlan.asInstanceOf[Project].projectList.forall(!_.deterministic)) + assert(df.head().getDouble(0) >= 0.0) + + val foo1 = udf(() => { Math.random() }).nonDeterministic() + val df1 = testData.select(foo1()) + assert(df1.logicalPlan.asInstanceOf[Project].projectList.forall(!_.deterministic)) + assert(df1.head().getDouble(0) >= 0.0) + + val foo2 = udf(() => { Math.random() }, DataTypes.DoubleType).nonDeterministic() + val df2 = testData.select(foo2()) + assert(df2.logicalPlan.asInstanceOf[Project].projectList.forall(!_.deterministic)) + assert(df2.head().getDouble(0) >= 0.0) } test("TwoArgument UDF") { From eb9a7fc28b0f9c15f39178aebfecaacece1e4523 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Thu, 13 Jul 2017 15:11:17 -0700 Subject: [PATCH 02/12] fix. --- .../test/java/test/org/apache/spark/sql/JavaRandUDF.java | 2 +- .../test/java/test/org/apache/spark/sql/JavaUDFSuite.java | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaRandUDF.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaRandUDF.java index 5bf4d9f19700a..df806bc18288d 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaRandUDF.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaRandUDF.java @@ -27,4 +27,4 @@ public class JavaRandUDF implements UDF1 { public Double call(Integer i) { return i + Math.random(); } -} \ No newline at end of file +} diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java index d46c4a7dc0ae9..a0ac7dafe7cbf 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java @@ -134,7 +134,7 @@ public Double call(Integer i) { @Test public void udf7Test() { spark.udf().registerJava( - "randUDF", randUDFTest.class.getName(), DataTypes.DoubleType, false, false); + "randUDF", randUDFTest.class.getName(), DataTypes.DoubleType, false, false); Row result = spark.sql("SELECT randUDF(1)").head(); Assert.assertTrue(result.getDouble(0) >= 0.0); @@ -144,11 +144,11 @@ public void udf7Test() { @Test public void udf8Test() { spark.udf().register( - "randUDF", (Integer i) -> i + Math.random(), DataTypes.DoubleType, false, false); + "randUDF", (Integer i) -> i + Math.random(), DataTypes.DoubleType, false, false); Row result = spark.sql("SELECT randUDF(1)").head(); Assert.assertTrue(result.getDouble(0) >= 0.0); Assert.assertTrue("EXPLAIN outputs are expected to contain the UDF name.", - spark.sql("EXPLAIN SELECT randUDF(1) AS r").collectAsList().toString().contains("randUDF")); + spark.sql("EXPLAIN SELECT randUDF(1) AS r").collectAsList().toString().contains("randUDF")); } } From a336ffc1bd70050d82eef79f6179e25231b23aa1 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Sun, 16 Jul 2017 21:25:32 -0700 Subject: [PATCH 03/12] fix. --- python/pyspark/sql/context.py | 7 +- .../sql/catalyst/analysis/Analyzer.scala | 2 +- .../sql/catalyst/expressions/ScalaUDF.scala | 5 +- .../apache/spark/sql/UDFRegistration.scala | 437 ++++++++---------- .../sql/expressions/UserDefinedFunction.scala | 20 +- .../org/apache/spark/sql/JavaUDFSuite.java | 8 +- .../scala/org/apache/spark/sql/UDFSuite.scala | 3 +- 7 files changed, 201 insertions(+), 281 deletions(-) diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index a0b1ab6d7ec9b..19f10e98e0526 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -208,8 +208,7 @@ def registerFunction(self, name, f, returnType=StringType()): @ignore_unicode_prefix @since(2.1) - def registerJavaFunction(self, name, javaClassName, returnType=None, deterministic=True, - distinctLike=False): + def registerJavaFunction(self, name, javaClassName, returnType=None, deterministic=True): """Register a java UDF so it can be used in SQL statements. In addition to a name and the function itself, the return type can be optionally specified. @@ -220,8 +219,6 @@ def registerJavaFunction(self, name, javaClassName, returnType=None, determinist :param returnType: a :class:`pyspark.sql.types.DataType` object. :param deterministic: a flag indicating if the UDF is deterministic. Deterministic UDF returns same result each time it is invoked with a particular input. - :param distinctLike: a UDF is considered distinctLike if the UDF can be evaluated on just - the distinct values of a column. >>> sqlContext.registerJavaFunction("javaStringLength", ... "test.org.apache.spark.sql.JavaStringLength", IntegerType()) @@ -241,7 +238,7 @@ def registerJavaFunction(self, name, javaClassName, returnType=None, determinist if returnType is not None: jdt = self.sparkSession._jsparkSession.parseDataType(returnType.json()) self.sparkSession._jsparkSession.udf().registerJava( - name, javaClassName, jdt, deterministic, distinctLike) + name, javaClassName, jdt, deterministic) @ignore_unicode_prefix @since(2.3) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index c0e1c2e208033..33f9e9543bbdd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1950,7 +1950,7 @@ class Analyzer( case p => p transformExpressionsUp { - case udf @ ScalaUDF(func, _, inputs, _, _, _, _, _) => + case udf @ ScalaUDF(func, _, inputs, _, _, _, _) => val parameterTypes = ScalaReflection.getParameterTypes(func) assert(parameterTypes.length == inputs.length) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala index ced066b052fc2..9df0e2e1415c0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala @@ -38,8 +38,6 @@ import org.apache.spark.sql.types.DataType * @param nullable True if the UDF can return null value. * @param udfDeterministic True if the UDF is deterministic. Deterministic UDF returns same result * each time it is invoked with a particular input. - * @param distinctLike A UDF is considered distinctLike if the UDF can be evaluated on just the - * distinct values of a column. */ case class ScalaUDF( function: AnyRef, @@ -48,8 +46,7 @@ case class ScalaUDF( inputTypes: Seq[DataType] = Nil, udfName: Option[String] = None, nullable: Boolean = true, - udfDeterministic: Boolean = true, - distinctLike: Boolean = false) + udfDeterministic: Boolean = true) extends Expression with ImplicitCastInputTypes with NonSQLExpression { override def deterministic: Boolean = udfDeterministic && children.forall(_.deterministic) 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 e331f8faa2358..38b7a0d67b6b3 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 @@ -109,7 +109,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register[$typeTags](name: String, func: Function$x[$types]): UserDefinedFunction = { - register(name, func, deterministic = true, distinctLike = false) + register(name, func, deterministic = true) } /** @@ -117,20 +117,18 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @tparam RT return type of UDF. * @since 2.3.0 */ - def register[$typeTags](name: String, func: Function$x[$types], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { + def register[$typeTags](name: String, func: Function$x[$types], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputTypes = Try($inputTypes).toOption def builder(e: Seq[Expression]) = if (e.length == $x) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: $x; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf - val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism - withDistinctLike + if (!deterministic) udf.nonDeterministic() else udf }""") } @@ -147,17 +145,17 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends | * @since $version | */ |def register(name: String, f: UDF$i[$extTypeArgs], returnType: DataType): Unit = { - | register(name, f, returnType, deterministic = true, distinctLike = false) + | register(name, f, returnType, deterministic = true) |} | |/** | * Registers a user-defined function with ${i} arguments. | * @since 2.3.0 | */ - |def register(name: String, f: UDF$i[$extTypeArgs], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { + |def register(name: String, f: UDF$i[$extTypeArgs], returnType: DataType, deterministic: Boolean): Unit = { | val func = f$anyCast.call($anyParams) | def builder(e: Seq[Expression]) = if (e.length == $i) { - | ScalaUDF($funcCall, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) + | ScalaUDF($funcCall, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) | } else { | throw new AnalysisException("Invalid number of arguments for function " + name + | ". Expected: $i; Found: " + e.length) @@ -173,7 +171,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register[RT: TypeTag](name: String, func: Function0[RT]): UserDefinedFunction = { - register(name, func, deterministic = true, distinctLike = false) + register(name, func, deterministic = true) } /** @@ -181,20 +179,18 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @tparam RT return type of UDF. * @since 2.3.0 */ - def register[RT: TypeTag](name: String, func: Function0[RT], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { + def register[RT: TypeTag](name: String, func: Function0[RT], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputTypes = Try(Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 0) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 0; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf - val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism - withDistinctLike + if (!deterministic) udf.nonDeterministic() else udf } /** @@ -203,7 +199,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag](name: String, func: Function1[A1, RT]): UserDefinedFunction = { - register(name, func, deterministic = true, distinctLike = false) + register(name, func, deterministic = true) } /** @@ -211,20 +207,18 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @tparam RT return type of UDF. * @since 2.3.0 */ - def register[RT: TypeTag, A1: TypeTag](name: String, func: Function1[A1, RT], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { + def register[RT: TypeTag, A1: TypeTag](name: String, func: Function1[A1, RT], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 1) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 1; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf - val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism - withDistinctLike + if (!deterministic) udf.nonDeterministic() else udf } /** @@ -233,7 +227,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag](name: String, func: Function2[A1, A2, RT]): UserDefinedFunction = { - register(name, func, deterministic = true, distinctLike = false) + register(name, func, deterministic = true) } /** @@ -241,20 +235,18 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @tparam RT return type of UDF. * @since 2.3.0 */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag](name: String, func: Function2[A1, A2, RT], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag](name: String, func: Function2[A1, A2, RT], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 2) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 2; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf - val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism - withDistinctLike + if (!deterministic) udf.nonDeterministic() else udf } /** @@ -263,7 +255,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag](name: String, func: Function3[A1, A2, A3, RT]): UserDefinedFunction = { - register(name, func, deterministic = true, distinctLike = false) + register(name, func, deterministic = true) } /** @@ -271,20 +263,18 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @tparam RT return type of UDF. * @since 2.3.0 */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag](name: String, func: Function3[A1, A2, A3, RT], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag](name: String, func: Function3[A1, A2, A3, RT], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 3) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 3; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf - val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism - withDistinctLike + if (!deterministic) udf.nonDeterministic() else udf } /** @@ -293,7 +283,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag](name: String, func: Function4[A1, A2, A3, A4, RT]): UserDefinedFunction = { - register(name, func, deterministic = true, distinctLike = false) + register(name, func, deterministic = true) } /** @@ -301,20 +291,18 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @tparam RT return type of UDF. * @since 2.3.0 */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag](name: String, func: Function4[A1, A2, A3, A4, RT], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag](name: String, func: Function4[A1, A2, A3, A4, RT], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 4) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 4; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf - val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism - withDistinctLike + if (!deterministic) udf.nonDeterministic() else udf } /** @@ -323,7 +311,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ 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 = { - register(name, func, deterministic = true, distinctLike = false) + register(name, func, deterministic = true) } /** @@ -331,20 +319,18 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @tparam RT return type of UDF. * @since 2.3.0 */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag](name: String, func: Function5[A1, A2, A3, A4, A5, RT], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag](name: String, func: Function5[A1, A2, A3, A4, A5, RT], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 5) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 5; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf - val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism - withDistinctLike + if (!deterministic) udf.nonDeterministic() else udf } /** @@ -353,7 +339,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ 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 = { - register(name, func, deterministic = true, distinctLike = false) + register(name, func, deterministic = true) } /** @@ -361,20 +347,18 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @tparam RT return type of UDF. * @since 2.3.0 */ - 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], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { + 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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 6) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 6; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf - val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism - withDistinctLike + if (!deterministic) udf.nonDeterministic() else udf } /** @@ -383,7 +367,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ 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 = { - register(name, func, deterministic = true, distinctLike = false) + register(name, func, deterministic = true) } /** @@ -391,20 +375,18 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @tparam RT return type of UDF. * @since 2.3.0 */ - 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], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { + 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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 7) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 7; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf - val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism - withDistinctLike + if (!deterministic) udf.nonDeterministic() else udf } /** @@ -413,7 +395,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ 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 = { - register(name, func, deterministic = true, distinctLike = false) + register(name, func, deterministic = true) } /** @@ -421,20 +403,18 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @tparam RT return type of UDF. * @since 2.3.0 */ - 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], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { + 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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 8) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 8; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf - val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism - withDistinctLike + if (!deterministic) udf.nonDeterministic() else udf } /** @@ -443,7 +423,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ 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 = { - register(name, func, deterministic = true, distinctLike = false) + register(name, func, deterministic = true) } /** @@ -451,20 +431,18 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @tparam RT return type of UDF. * @since 2.3.0 */ - 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], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { + 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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 9) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 9; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf - val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism - withDistinctLike + if (!deterministic) udf.nonDeterministic() else udf } /** @@ -473,7 +451,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ 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 = { - register(name, func, deterministic = true, distinctLike = false) + register(name, func, deterministic = true) } /** @@ -481,20 +459,18 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @tparam RT return type of UDF. * @since 2.3.0 */ - 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], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { + 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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 10) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 10; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf - val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism - withDistinctLike + if (!deterministic) udf.nonDeterministic() else udf } /** @@ -503,7 +479,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ 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 = { - register(name, func, deterministic = true, distinctLike = false) + register(name, func, deterministic = true) } /** @@ -511,20 +487,18 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @tparam RT return type of UDF. * @since 2.3.0 */ - 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], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { + 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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 11) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 11; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf - val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism - withDistinctLike + if (!deterministic) udf.nonDeterministic() else udf } /** @@ -533,7 +507,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ 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 = { - register(name, func, deterministic = true, distinctLike = false) + register(name, func, deterministic = true) } /** @@ -541,20 +515,18 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @tparam RT return type of UDF. * @since 2.3.0 */ - 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], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { + 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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 12) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 12; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf - val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism - withDistinctLike + if (!deterministic) udf.nonDeterministic() else udf } /** @@ -563,7 +535,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ 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 = { - register(name, func, deterministic = true, distinctLike = false) + register(name, func, deterministic = true) } /** @@ -571,20 +543,18 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @tparam RT return type of UDF. * @since 2.3.0 */ - 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], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { + 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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 13) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 13; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf - val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism - withDistinctLike + if (!deterministic) udf.nonDeterministic() else udf } /** @@ -593,7 +563,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ 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 = { - register(name, func, deterministic = true, distinctLike = false) + register(name, func, deterministic = true) } /** @@ -601,20 +571,18 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @tparam RT return type of UDF. * @since 2.3.0 */ - 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], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { + 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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 14) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 14; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf - val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism - withDistinctLike + if (!deterministic) udf.nonDeterministic() else udf } /** @@ -623,7 +591,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ 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 = { - register(name, func, deterministic = true, distinctLike = false) + register(name, func, deterministic = true) } /** @@ -631,20 +599,18 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @tparam RT return type of UDF. * @since 2.3.0 */ - 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], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { + 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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 15) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 15; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf - val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism - withDistinctLike + if (!deterministic) udf.nonDeterministic() else udf } /** @@ -653,7 +619,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ 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 = { - register(name, func, deterministic = true, distinctLike = false) + register(name, func, deterministic = true) } /** @@ -661,20 +627,18 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @tparam RT return type of UDF. * @since 2.3.0 */ - 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], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { + 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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 16) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 16; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf - val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism - withDistinctLike + if (!deterministic) udf.nonDeterministic() else udf } /** @@ -683,7 +647,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ 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 = { - register(name, func, deterministic = true, distinctLike = false) + register(name, func, deterministic = true) } /** @@ -691,20 +655,18 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @tparam RT return type of UDF. * @since 2.3.0 */ - 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], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { + 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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 17) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 17; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf - val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism - withDistinctLike + if (!deterministic) udf.nonDeterministic() else udf } /** @@ -713,7 +675,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ 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 = { - register(name, func, deterministic = true, distinctLike = false) + register(name, func, deterministic = true) } /** @@ -721,20 +683,18 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @tparam RT return type of UDF. * @since 2.3.0 */ - 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], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { + 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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 18) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 18; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf - val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism - withDistinctLike + if (!deterministic) udf.nonDeterministic() else udf } /** @@ -743,7 +703,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ 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 = { - register(name, func, deterministic = true, distinctLike = false) + register(name, func, deterministic = true) } /** @@ -751,20 +711,18 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @tparam RT return type of UDF. * @since 2.3.0 */ - 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], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { + 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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 19) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 19; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf - val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism - withDistinctLike + if (!deterministic) udf.nonDeterministic() else udf } /** @@ -773,7 +731,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ 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 = { - register(name, func, deterministic = true, distinctLike = false) + register(name, func, deterministic = true) } /** @@ -781,20 +739,18 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @tparam RT return type of UDF. * @since 2.3.0 */ - 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], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { + 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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 20) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 20; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf - val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism - withDistinctLike + if (!deterministic) udf.nonDeterministic() else udf } /** @@ -803,7 +759,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ 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 = { - register(name, func, deterministic = true, distinctLike = false) + register(name, func, deterministic = true) } /** @@ -811,20 +767,18 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @tparam RT return type of UDF. * @since 2.3.0 */ - 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], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { + 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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 21) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 21; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf - val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism - withDistinctLike + if (!deterministic) udf.nonDeterministic() else udf } /** @@ -833,7 +787,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ 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 = { - register(name, func, deterministic = true, distinctLike = false) + register(name, func, deterministic = true) } /** @@ -841,20 +795,18 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @tparam RT return type of UDF. * @since 2.3.0 */ - 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], deterministic: Boolean, distinctLike: Boolean): UserDefinedFunction = { + 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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 22) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 22; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - val withDeterminism = if (!deterministic) udf.nonDeterministic() else udf - val withDistinctLike = if (distinctLike) withDeterminism.withDistinctLike() else withDeterminism - withDistinctLike + if (!deterministic) udf.nonDeterministic() else udf } ////////////////////////////////////////////////////////////////////////////////////////////// @@ -869,7 +821,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * via reflection. */ private[sql] def registerJava(name: String, className: String, returnDataType: DataType): Unit = { - registerJava(name, className, returnDataType, deterministic = true, distinctLike = false) + registerJava(name, className, returnDataType, deterministic = true) } /** * Registers a Java UDF class using reflection, for use from pyspark @@ -880,15 +832,12 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * via reflection. * @param deterministic True if the UDF is deterministic. Deterministic UDF returns same result * each time it is invoked with a particular input. - * @param distinctLike A UDF is considered distinctLike if the UDF can be evaluated on just the - * distinct values of a column. */ private[sql] def registerJava( name: String, className: String, returnDataType: DataType, - deterministic: Boolean, - distinctLike: Boolean): Unit = { + deterministic: Boolean): Unit = { try { val clazz = Utils.classForName(className) val udfInterfaces = clazz.getGenericInterfaces @@ -909,29 +858,29 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } udfInterfaces(0).getActualTypeArguments.length match { - case 1 => register(name, udf.asInstanceOf[UDF0[_]], returnType, deterministic, distinctLike) - case 2 => register(name, udf.asInstanceOf[UDF1[_, _]], returnType, deterministic, distinctLike) - case 3 => register(name, udf.asInstanceOf[UDF2[_, _, _]], returnType, deterministic, distinctLike) - case 4 => register(name, udf.asInstanceOf[UDF3[_, _, _, _]], returnType, deterministic, distinctLike) - case 5 => register(name, udf.asInstanceOf[UDF4[_, _, _, _, _]], returnType, deterministic, distinctLike) - case 6 => register(name, udf.asInstanceOf[UDF5[_, _, _, _, _, _]], returnType, deterministic, distinctLike) - case 7 => register(name, udf.asInstanceOf[UDF6[_, _, _, _, _, _, _]], returnType, deterministic, distinctLike) - case 8 => register(name, udf.asInstanceOf[UDF7[_, _, _, _, _, _, _, _]], returnType, deterministic, distinctLike) - case 9 => register(name, udf.asInstanceOf[UDF8[_, _, _, _, _, _, _, _, _]], returnType, deterministic, distinctLike) - case 10 => register(name, udf.asInstanceOf[UDF9[_, _, _, _, _, _, _, _, _, _]], returnType, deterministic, distinctLike) - case 11 => register(name, udf.asInstanceOf[UDF10[_, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic, distinctLike) - case 12 => register(name, udf.asInstanceOf[UDF11[_, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic, distinctLike) - case 13 => register(name, udf.asInstanceOf[UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic, distinctLike) - case 14 => register(name, udf.asInstanceOf[UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic, distinctLike) - case 15 => register(name, udf.asInstanceOf[UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic, distinctLike) - case 16 => register(name, udf.asInstanceOf[UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic, distinctLike) - case 17 => register(name, udf.asInstanceOf[UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic, distinctLike) - case 18 => register(name, udf.asInstanceOf[UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic, distinctLike) - case 19 => register(name, udf.asInstanceOf[UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic, distinctLike) - case 20 => register(name, udf.asInstanceOf[UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic, distinctLike) - case 21 => register(name, udf.asInstanceOf[UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic, distinctLike) - case 22 => register(name, udf.asInstanceOf[UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic, distinctLike) - case 23 => register(name, udf.asInstanceOf[UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic, distinctLike) + case 1 => register(name, udf.asInstanceOf[UDF0[_]], returnType, deterministic) + case 2 => register(name, udf.asInstanceOf[UDF1[_, _]], returnType, deterministic) + case 3 => register(name, udf.asInstanceOf[UDF2[_, _, _]], returnType, deterministic) + case 4 => register(name, udf.asInstanceOf[UDF3[_, _, _, _]], returnType, deterministic) + case 5 => register(name, udf.asInstanceOf[UDF4[_, _, _, _, _]], returnType, deterministic) + case 6 => register(name, udf.asInstanceOf[UDF5[_, _, _, _, _, _]], returnType, deterministic) + case 7 => register(name, udf.asInstanceOf[UDF6[_, _, _, _, _, _, _]], returnType, deterministic) + case 8 => register(name, udf.asInstanceOf[UDF7[_, _, _, _, _, _, _, _]], returnType, deterministic) + case 9 => register(name, udf.asInstanceOf[UDF8[_, _, _, _, _, _, _, _, _]], returnType, deterministic) + case 10 => register(name, udf.asInstanceOf[UDF9[_, _, _, _, _, _, _, _, _, _]], returnType, deterministic) + case 11 => register(name, udf.asInstanceOf[UDF10[_, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic) + case 12 => register(name, udf.asInstanceOf[UDF11[_, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic) + case 13 => register(name, udf.asInstanceOf[UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic) + case 14 => register(name, udf.asInstanceOf[UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic) + case 15 => register(name, udf.asInstanceOf[UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic) + case 16 => register(name, udf.asInstanceOf[UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic) + case 17 => register(name, udf.asInstanceOf[UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic) + case 18 => register(name, udf.asInstanceOf[UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic) + case 19 => register(name, udf.asInstanceOf[UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic) + case 20 => register(name, udf.asInstanceOf[UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic) + case 21 => register(name, udf.asInstanceOf[UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic) + case 22 => register(name, udf.asInstanceOf[UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic) + case 23 => register(name, udf.asInstanceOf[UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic) case n => throw new AnalysisException(s"UDF class with $n type arguments is not supported.") } @@ -973,17 +922,17 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 2.3.0 */ def register(name: String, f: UDF0[_], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true, distinctLike = false) + register(name, f, returnType, deterministic = true) } /** * Registers a user-defined function with 0 arguments. * @since 2.3.0 */ - def register(name: String, f: UDF0[_], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { + def register(name: String, f: UDF0[_], returnType: DataType, deterministic: Boolean): Unit = { val func = f.asInstanceOf[UDF0[Any]].call() def builder(e: Seq[Expression]) = if (e.length == 0) { - ScalaUDF(() => func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) + ScalaUDF(() => func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 0; Found: " + e.length) @@ -996,17 +945,17 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF1[_, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true, distinctLike = false) + register(name, f, returnType, deterministic = true) } /** * Registers a user-defined function with 1 arguments. * @since 2.3.0 */ - def register(name: String, f: UDF1[_, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { + def register(name: String, f: UDF1[_, _], returnType: DataType, deterministic: Boolean): Unit = { val func = f.asInstanceOf[UDF1[Any, Any]].call(_: Any) def builder(e: Seq[Expression]) = if (e.length == 1) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 1; Found: " + e.length) @@ -1019,17 +968,17 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF2[_, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true, distinctLike = false) + register(name, f, returnType, deterministic = true) } /** * Registers a user-defined function with 2 arguments. * @since 2.3.0 */ - def register(name: String, f: UDF2[_, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { + def register(name: String, f: UDF2[_, _, _], returnType: DataType, deterministic: Boolean): Unit = { val func = f.asInstanceOf[UDF2[Any, Any, Any]].call(_: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 2) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 2; Found: " + e.length) @@ -1042,17 +991,17 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF3[_, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true, distinctLike = false) + register(name, f, returnType, deterministic = true) } /** * Registers a user-defined function with 3 arguments. * @since 2.3.0 */ - def register(name: String, f: UDF3[_, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { + def register(name: String, f: UDF3[_, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { val func = f.asInstanceOf[UDF3[Any, Any, Any, Any]].call(_: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 3) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 3; Found: " + e.length) @@ -1065,17 +1014,17 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF4[_, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true, distinctLike = false) + register(name, f, returnType, deterministic = true) } /** * Registers a user-defined function with 4 arguments. * @since 2.3.0 */ - def register(name: String, f: UDF4[_, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { + def register(name: String, f: UDF4[_, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { val func = f.asInstanceOf[UDF4[Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 4) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 4; Found: " + e.length) @@ -1088,17 +1037,17 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF5[_, _, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true, distinctLike = false) + register(name, f, returnType, deterministic = true) } /** * Registers a user-defined function with 5 arguments. * @since 2.3.0 */ - def register(name: String, f: UDF5[_, _, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { + def register(name: String, f: UDF5[_, _, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { val func = f.asInstanceOf[UDF5[Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 5) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 5; Found: " + e.length) @@ -1111,17 +1060,17 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF6[_, _, _, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true, distinctLike = false) + register(name, f, returnType, deterministic = true) } /** * Registers a user-defined function with 6 arguments. * @since 2.3.0 */ - def register(name: String, f: UDF6[_, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { + def register(name: String, f: UDF6[_, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { val func = f.asInstanceOf[UDF6[Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 6) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 6; Found: " + e.length) @@ -1134,17 +1083,17 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF7[_, _, _, _, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true, distinctLike = false) + register(name, f, returnType, deterministic = true) } /** * Registers a user-defined function with 7 arguments. * @since 2.3.0 */ - def register(name: String, f: UDF7[_, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { + def register(name: String, f: UDF7[_, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { val func = f.asInstanceOf[UDF7[Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 7) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 7; Found: " + e.length) @@ -1157,17 +1106,17 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF8[_, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true, distinctLike = false) + register(name, f, returnType, deterministic = true) } /** * Registers a user-defined function with 8 arguments. * @since 2.3.0 */ - def register(name: String, f: UDF8[_, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { + def register(name: String, f: UDF8[_, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { val func = f.asInstanceOf[UDF8[Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 8) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 8; Found: " + e.length) @@ -1180,17 +1129,17 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF9[_, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true, distinctLike = false) + register(name, f, returnType, deterministic = true) } /** * Registers a user-defined function with 9 arguments. * @since 2.3.0 */ - def register(name: String, f: UDF9[_, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { + def register(name: String, f: UDF9[_, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 9) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 9; Found: " + e.length) @@ -1203,17 +1152,17 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF10[_, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true, distinctLike = false) + register(name, f, returnType, deterministic = true) } /** * Registers a user-defined function with 10 arguments. * @since 2.3.0 */ - def register(name: String, f: UDF10[_, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { + def register(name: String, f: UDF10[_, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 10) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 10; Found: " + e.length) @@ -1226,17 +1175,17 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF11[_, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true, distinctLike = false) + register(name, f, returnType, deterministic = true) } /** * Registers a user-defined function with 11 arguments. * @since 2.3.0 */ - def register(name: String, f: UDF11[_, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { + def register(name: String, f: UDF11[_, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 11) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 11; Found: " + e.length) @@ -1249,17 +1198,17 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true, distinctLike = false) + register(name, f, returnType, deterministic = true) } /** * Registers a user-defined function with 12 arguments. * @since 2.3.0 */ - def register(name: String, f: UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { + def register(name: String, f: UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 12) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 12; Found: " + e.length) @@ -1272,17 +1221,17 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true, distinctLike = false) + register(name, f, returnType, deterministic = true) } /** * Registers a user-defined function with 13 arguments. * @since 2.3.0 */ - def register(name: String, f: UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { + def register(name: String, f: UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 13) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 13; Found: " + e.length) @@ -1295,17 +1244,17 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true, distinctLike = false) + register(name, f, returnType, deterministic = true) } /** * Registers a user-defined function with 14 arguments. * @since 2.3.0 */ - def register(name: String, f: UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { + def register(name: String, f: UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 14) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 14; Found: " + e.length) @@ -1318,17 +1267,17 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true, distinctLike = false) + register(name, f, returnType, deterministic = true) } /** * Registers a user-defined function with 15 arguments. * @since 2.3.0 */ - def register(name: String, f: UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { + def register(name: String, f: UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 15) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 15; Found: " + e.length) @@ -1341,17 +1290,17 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true, distinctLike = false) + register(name, f, returnType, deterministic = true) } /** * Registers a user-defined function with 16 arguments. * @since 2.3.0 */ - def register(name: String, f: UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { + def register(name: String, f: UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 16) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 16; Found: " + e.length) @@ -1364,17 +1313,17 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true, distinctLike = false) + register(name, f, returnType, deterministic = true) } /** * Registers a user-defined function with 17 arguments. * @since 2.3.0 */ - def register(name: String, f: UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { + def register(name: String, f: UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 17) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 17; Found: " + e.length) @@ -1387,17 +1336,17 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true, distinctLike = false) + register(name, f, returnType, deterministic = true) } /** * Registers a user-defined function with 18 arguments. * @since 2.3.0 */ - def register(name: String, f: UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { + def register(name: String, f: UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 18) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 18; Found: " + e.length) @@ -1410,17 +1359,17 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true, distinctLike = false) + register(name, f, returnType, deterministic = true) } /** * Registers a user-defined function with 19 arguments. * @since 2.3.0 */ - def register(name: String, f: UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { + def register(name: String, f: UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 19) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 19; Found: " + e.length) @@ -1433,17 +1382,17 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true, distinctLike = false) + register(name, f, returnType, deterministic = true) } /** * Registers a user-defined function with 20 arguments. * @since 2.3.0 */ - def register(name: String, f: UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { + def register(name: String, f: UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 20) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 20; Found: " + e.length) @@ -1456,17 +1405,17 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true, distinctLike = false) + register(name, f, returnType, deterministic = true) } /** * Registers a user-defined function with 21 arguments. * @since 2.3.0 */ - def register(name: String, f: UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { + def register(name: String, f: UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 21) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 21; Found: " + e.length) @@ -1479,17 +1428,17 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true, distinctLike = false) + register(name, f, returnType, deterministic = true) } /** * Registers a user-defined function with 22 arguments. * @since 2.3.0 */ - def register(name: String, f: UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean, distinctLike: Boolean): Unit = { + def register(name: String, f: UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 22) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic, distinctLike) + ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 22; Found: " + e.length) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala index a6aada020cf1c..433c9ac8d0b8b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala @@ -46,7 +46,6 @@ case class UserDefinedFunction protected[sql] ( private var _nameOption: Option[String] = None private var _nullable: Boolean = true private var _deterministic: Boolean = true - private var _distinctLike: Boolean = false /** * Returns true when the UDF can return a nullable value. @@ -75,8 +74,7 @@ case class UserDefinedFunction protected[sql] ( inputTypes.getOrElse(Nil), udfName = _nameOption, nullable = _nullable, - udfDeterministic = _deterministic, - distinctLike = _distinctLike)) + udfDeterministic = _deterministic)) } private def copyAll(): UserDefinedFunction = { @@ -84,7 +82,6 @@ case class UserDefinedFunction protected[sql] ( udf._nameOption = _nameOption udf._nullable = _nullable udf._deterministic = _deterministic - udf._distinctLike = _distinctLike udf } @@ -128,19 +125,4 @@ case class UserDefinedFunction protected[sql] ( udf } } - - /** - * Updates UserDefinedFunction to distinctLike. - * - * @since 2.3.0 - */ - def withDistinctLike(): UserDefinedFunction = { - if (_distinctLike) { - this - } else { - val udf = copyAll() - udf._distinctLike = true - udf - } - } } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java index a0ac7dafe7cbf..7e8c1c88232ee 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java @@ -133,9 +133,7 @@ public Double call(Integer i) { @SuppressWarnings("unchecked") @Test public void udf7Test() { - spark.udf().registerJava( - "randUDF", randUDFTest.class.getName(), DataTypes.DoubleType, false, false); - + spark.udf().registerJava("randUDF", randUDFTest.class.getName(), DataTypes.DoubleType, false); Row result = spark.sql("SELECT randUDF(1)").head(); Assert.assertTrue(result.getDouble(0) >= 0.0); } @@ -143,9 +141,7 @@ public void udf7Test() { @SuppressWarnings("unchecked") @Test public void udf8Test() { - spark.udf().register( - "randUDF", (Integer i) -> i + Math.random(), DataTypes.DoubleType, false, false); - + spark.udf().register("randUDF", (Integer i) -> i + Math.random(), DataTypes.DoubleType, false); Row result = spark.sql("SELECT randUDF(1)").head(); Assert.assertTrue(result.getDouble(0) >= 0.0); Assert.assertTrue("EXPLAIN outputs are expected to contain the UDF name.", 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 d31d6937980be..7146de740ac37 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 @@ -113,8 +113,7 @@ class UDFSuite extends QueryTest with SharedSQLContext { } test("ZeroArgument non-deterministic UDF") { - spark.udf.register( - "random0", () => { Math.random() }, deterministic = false, distinctLike = false) + spark.udf.register("random0", () => { Math.random() }, deterministic = false) val df = sql("SELECT random0()") assert(df.logicalPlan.asInstanceOf[Project].projectList.forall(!_.deterministic)) assert(df.head().getDouble(0) >= 0.0) From 9db97cbf0df28d6058f80b879d37641fdddb56f9 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Tue, 18 Jul 2017 13:48:49 -0700 Subject: [PATCH 04/12] fix. --- .../apache/spark/sql/UDFRegistration.scala | 244 ++++++++++++------ .../apache/spark/sql/SQLContextSuite.scala | 2 +- .../scala/org/apache/spark/sql/UDFSuite.scala | 2 +- 3 files changed, 172 insertions(+), 76 deletions(-) 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 38b7a0d67b6b3..07efdd0fec6ba 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 @@ -109,15 +109,19 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register[$typeTags](name: String, func: Function$x[$types]): UserDefinedFunction = { - register(name, func, deterministic = true) + registerUDF(name, func, deterministic = true) } /** - * Registers a Scala closure of ${x} arguments as user-defined function (UDF). + * Registers a non-deterministic Scala closure of ${x} arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 2.3.0 */ - def register[$typeTags](name: String, func: Function$x[$types], deterministic: Boolean): UserDefinedFunction = { + def registerNonDeterministic[$typeTags](name: String, func: Function$x[$types]): UserDefinedFunction = { + registerUDF(name, func, deterministic = false) + } + + private def registerUDF[$typeTags](name: String, func: Function$x[$types], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputTypes = Try($inputTypes).toOption def builder(e: Seq[Expression]) = if (e.length == $x) { @@ -145,14 +149,14 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends | * @since $version | */ |def register(name: String, f: UDF$i[$extTypeArgs], returnType: DataType): Unit = { - | register(name, f, returnType, deterministic = true) + | registerNonDeterministic(name, f, returnType, deterministic = true) |} | |/** | * Registers a user-defined function with ${i} arguments. | * @since 2.3.0 | */ - |def register(name: String, f: UDF$i[$extTypeArgs], returnType: DataType, deterministic: Boolean): Unit = { + |def registerNonDeterministic(name: String, f: UDF$i[$extTypeArgs], returnType: DataType, deterministic: Boolean): Unit = { | val func = f$anyCast.call($anyParams) | def builder(e: Seq[Expression]) = if (e.length == $i) { | ScalaUDF($funcCall, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) @@ -171,15 +175,19 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register[RT: TypeTag](name: String, func: Function0[RT]): UserDefinedFunction = { - register(name, func, deterministic = true) + registerUDF(name, func, deterministic = true) } /** - * Registers a Scala closure of 0 arguments as user-defined function (UDF). + * Registers a non-deterministic Scala closure of 0 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 2.3.0 */ - def register[RT: TypeTag](name: String, func: Function0[RT], deterministic: Boolean): UserDefinedFunction = { + def registerNonDeterministic[RT: TypeTag](name: String, func: Function0[RT]): UserDefinedFunction = { + registerUDF(name, func, deterministic = false) + } + + private def registerUDF[RT: TypeTag](name: String, func: Function0[RT], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputTypes = Try(Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 0) { @@ -199,15 +207,19 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag](name: String, func: Function1[A1, RT]): UserDefinedFunction = { - register(name, func, deterministic = true) + registerUDF(name, func, deterministic = true) } /** - * Registers a Scala closure of 1 arguments as user-defined function (UDF). + * Registers a non-deterministic Scala closure of 1 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 2.3.0 */ - def register[RT: TypeTag, A1: TypeTag](name: String, func: Function1[A1, RT], deterministic: Boolean): UserDefinedFunction = { + def registerNonDeterministic[RT: TypeTag, A1: TypeTag](name: String, func: Function1[A1, RT]): UserDefinedFunction = { + registerUDF(name, func, deterministic = false) + } + + private def registerUDF[RT: TypeTag, A1: TypeTag](name: String, func: Function1[A1, RT], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 1) { @@ -227,15 +239,19 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag](name: String, func: Function2[A1, A2, RT]): UserDefinedFunction = { - register(name, func, deterministic = true) + registerUDF(name, func, deterministic = true) } /** - * Registers a Scala closure of 2 arguments as user-defined function (UDF). + * Registers a non-deterministic Scala closure of 2 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 2.3.0 */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag](name: String, func: Function2[A1, A2, RT], deterministic: Boolean): UserDefinedFunction = { + def registerNonDeterministic[RT: TypeTag, A1: TypeTag, A2: TypeTag](name: String, func: Function2[A1, A2, RT]): UserDefinedFunction = { + registerUDF(name, func, deterministic = false) + } + + private def registerUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag](name: String, func: Function2[A1, A2, RT], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 2) { @@ -255,15 +271,19 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag](name: String, func: Function3[A1, A2, A3, RT]): UserDefinedFunction = { - register(name, func, deterministic = true) + registerUDF(name, func, deterministic = true) } /** - * Registers a Scala closure of 3 arguments as user-defined function (UDF). + * Registers a non-deterministic Scala closure of 3 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 2.3.0 */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag](name: String, func: Function3[A1, A2, A3, RT], deterministic: Boolean): UserDefinedFunction = { + def registerNonDeterministic[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag](name: String, func: Function3[A1, A2, A3, RT]): UserDefinedFunction = { + registerUDF(name, func, deterministic = false) + } + + private def registerUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag](name: String, func: Function3[A1, A2, A3, RT], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 3) { @@ -283,15 +303,19 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag](name: String, func: Function4[A1, A2, A3, A4, RT]): UserDefinedFunction = { - register(name, func, deterministic = true) + registerUDF(name, func, deterministic = true) } /** - * Registers a Scala closure of 4 arguments as user-defined function (UDF). + * Registers a non-deterministic Scala closure of 4 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 2.3.0 */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag](name: String, func: Function4[A1, A2, A3, A4, RT], deterministic: Boolean): UserDefinedFunction = { + def registerNonDeterministic[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag](name: String, func: Function4[A1, A2, A3, A4, RT]): UserDefinedFunction = { + registerUDF(name, func, deterministic = false) + } + + private def registerUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag](name: String, func: Function4[A1, A2, A3, A4, RT], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 4) { @@ -311,15 +335,19 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ 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 = { - register(name, func, deterministic = true) + registerUDF(name, func, deterministic = true) } /** - * Registers a Scala closure of 5 arguments as user-defined function (UDF). + * Registers a non-deterministic Scala closure of 5 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 2.3.0 */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag](name: String, func: Function5[A1, A2, A3, A4, A5, RT], deterministic: Boolean): UserDefinedFunction = { + def registerNonDeterministic[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag](name: String, func: Function5[A1, A2, A3, A4, A5, RT]): UserDefinedFunction = { + registerUDF(name, func, deterministic = false) + } + + private def registerUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag](name: String, func: Function5[A1, A2, A3, A4, A5, RT], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 5) { @@ -339,15 +367,19 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ 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 = { - register(name, func, deterministic = true) + registerUDF(name, func, deterministic = true) } /** - * Registers a Scala closure of 6 arguments as user-defined function (UDF). + * Registers a non-deterministic Scala closure of 6 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 2.3.0 */ - 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], deterministic: Boolean): UserDefinedFunction = { + def registerNonDeterministic[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 = { + registerUDF(name, func, deterministic = false) + } + + private def registerUDF[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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 6) { @@ -367,15 +399,19 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ 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 = { - register(name, func, deterministic = true) + registerUDF(name, func, deterministic = true) } /** - * Registers a Scala closure of 7 arguments as user-defined function (UDF). + * Registers a non-deterministic Scala closure of 7 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 2.3.0 */ - 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], deterministic: Boolean): UserDefinedFunction = { + def registerNonDeterministic[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 = { + registerUDF(name, func, deterministic = false) + } + + private def registerUDF[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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 7) { @@ -395,15 +431,19 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ 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 = { - register(name, func, deterministic = true) + registerUDF(name, func, deterministic = true) } /** - * Registers a Scala closure of 8 arguments as user-defined function (UDF). + * Registers a non-deterministic Scala closure of 8 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 2.3.0 */ - 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], deterministic: Boolean): UserDefinedFunction = { + def registerNonDeterministic[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 = { + registerUDF(name, func, deterministic = false) + } + + private def registerUDF[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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 8) { @@ -423,15 +463,19 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ 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 = { - register(name, func, deterministic = true) + registerUDF(name, func, deterministic = true) } /** - * Registers a Scala closure of 9 arguments as user-defined function (UDF). + * Registers a non-deterministic Scala closure of 9 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 2.3.0 */ - 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], deterministic: Boolean): UserDefinedFunction = { + def registerNonDeterministic[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 = { + registerUDF(name, func, deterministic = false) + } + + private def registerUDF[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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 9) { @@ -451,15 +495,19 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ 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 = { - register(name, func, deterministic = true) + registerUDF(name, func, deterministic = true) } /** - * Registers a Scala closure of 10 arguments as user-defined function (UDF). + * Registers a non-deterministic Scala closure of 10 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 2.3.0 */ - 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], deterministic: Boolean): UserDefinedFunction = { + def registerNonDeterministic[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 = { + registerUDF(name, func, deterministic = false) + } + + private def registerUDF[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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 10) { @@ -479,15 +527,19 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ 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 = { - register(name, func, deterministic = true) + registerUDF(name, func, deterministic = true) } /** - * Registers a Scala closure of 11 arguments as user-defined function (UDF). + * Registers a non-deterministic Scala closure of 11 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 2.3.0 */ - 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], deterministic: Boolean): UserDefinedFunction = { + def registerNonDeterministic[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 = { + registerUDF(name, func, deterministic = false) + } + + private def registerUDF[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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 11) { @@ -507,15 +559,19 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ 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 = { - register(name, func, deterministic = true) + registerUDF(name, func, deterministic = true) } /** - * Registers a Scala closure of 12 arguments as user-defined function (UDF). + * Registers a non-deterministic Scala closure of 12 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 2.3.0 */ - 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], deterministic: Boolean): UserDefinedFunction = { + def registerNonDeterministic[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 = { + registerUDF(name, func, deterministic = false) + } + + private def registerUDF[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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 12) { @@ -535,15 +591,19 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ 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 = { - register(name, func, deterministic = true) + registerUDF(name, func, deterministic = true) } /** - * Registers a Scala closure of 13 arguments as user-defined function (UDF). + * Registers a non-deterministic Scala closure of 13 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 2.3.0 */ - 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], deterministic: Boolean): UserDefinedFunction = { + def registerNonDeterministic[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 = { + registerUDF(name, func, deterministic = false) + } + + private def registerUDF[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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 13) { @@ -563,15 +623,19 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ 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 = { - register(name, func, deterministic = true) + registerUDF(name, func, deterministic = true) } /** - * Registers a Scala closure of 14 arguments as user-defined function (UDF). + * Registers a non-deterministic Scala closure of 14 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 2.3.0 */ - 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], deterministic: Boolean): UserDefinedFunction = { + def registerNonDeterministic[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 = { + registerUDF(name, func, deterministic = false) + } + + private def registerUDF[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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 14) { @@ -591,15 +655,19 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ 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 = { - register(name, func, deterministic = true) + registerUDF(name, func, deterministic = true) } /** - * Registers a Scala closure of 15 arguments as user-defined function (UDF). + * Registers a non-deterministic Scala closure of 15 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 2.3.0 */ - 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], deterministic: Boolean): UserDefinedFunction = { + def registerNonDeterministic[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 = { + registerUDF(name, func, deterministic = false) + } + + private def registerUDF[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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 15) { @@ -619,15 +687,19 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ 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 = { - register(name, func, deterministic = true) + registerUDF(name, func, deterministic = true) } /** - * Registers a Scala closure of 16 arguments as user-defined function (UDF). + * Registers a non-deterministic Scala closure of 16 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 2.3.0 */ - 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], deterministic: Boolean): UserDefinedFunction = { + def registerNonDeterministic[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 = { + registerUDF(name, func, deterministic = false) + } + + private def registerUDF[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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 16) { @@ -647,15 +719,19 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ 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 = { - register(name, func, deterministic = true) + registerUDF(name, func, deterministic = true) } /** - * Registers a Scala closure of 17 arguments as user-defined function (UDF). + * Registers a non-deterministic Scala closure of 17 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 2.3.0 */ - 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], deterministic: Boolean): UserDefinedFunction = { + def registerNonDeterministic[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 = { + registerUDF(name, func, deterministic = false) + } + + private def registerUDF[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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 17) { @@ -675,15 +751,19 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ 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 = { - register(name, func, deterministic = true) + registerUDF(name, func, deterministic = true) } /** - * Registers a Scala closure of 18 arguments as user-defined function (UDF). + * Registers a non-deterministic Scala closure of 18 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 2.3.0 */ - 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], deterministic: Boolean): UserDefinedFunction = { + def registerNonDeterministic[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 = { + registerUDF(name, func, deterministic = false) + } + + private def registerUDF[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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 18) { @@ -703,15 +783,19 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ 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 = { - register(name, func, deterministic = true) + registerUDF(name, func, deterministic = true) } /** - * Registers a Scala closure of 19 arguments as user-defined function (UDF). + * Registers a non-deterministic Scala closure of 19 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 2.3.0 */ - 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], deterministic: Boolean): UserDefinedFunction = { + def registerNonDeterministic[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 = { + registerUDF(name, func, deterministic = false) + } + + private def registerUDF[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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 19) { @@ -731,15 +815,19 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ 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 = { - register(name, func, deterministic = true) + registerUDF(name, func, deterministic = true) } /** - * Registers a Scala closure of 20 arguments as user-defined function (UDF). + * Registers a non-deterministic Scala closure of 20 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 2.3.0 */ - 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], deterministic: Boolean): UserDefinedFunction = { + def registerNonDeterministic[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 = { + registerUDF(name, func, deterministic = false) + } + + private def registerUDF[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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 20) { @@ -759,15 +847,19 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ 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 = { - register(name, func, deterministic = true) + registerUDF(name, func, deterministic = true) } /** - * Registers a Scala closure of 21 arguments as user-defined function (UDF). + * Registers a non-deterministic Scala closure of 21 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 2.3.0 */ - 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], deterministic: Boolean): UserDefinedFunction = { + def registerNonDeterministic[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 = { + registerUDF(name, func, deterministic = false) + } + + private def registerUDF[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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 21) { @@ -787,15 +879,19 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ 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 = { - register(name, func, deterministic = true) + registerUDF(name, func, deterministic = true) } /** - * Registers a Scala closure of 22 arguments as user-defined function (UDF). + * Registers a non-deterministic Scala closure of 22 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 2.3.0 */ - 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], deterministic: Boolean): UserDefinedFunction = { + def registerNonDeterministic[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 = { + registerUDF(name, func, deterministic = false) + } + + private def registerUDF[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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 22) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala index 7b91afd9d7577..2b35db411e2ab 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala @@ -69,7 +69,7 @@ class SQLContextSuite extends SparkFunSuite with SharedSparkContext { // UDF should not be shared def myadd(a: Int, b: Int): Int = a + b - session1.udf.register[Int, Int, Int]("myadd", myadd _) + session1.udf.register[Int, Int, Int]("myadd", myadd) session1.sql("select myadd(1, 2)").explain() intercept[AnalysisException] { session2.sql("select myadd(1, 2)").explain() 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 7146de740ac37..be4400ca191f1 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 @@ -113,7 +113,7 @@ class UDFSuite extends QueryTest with SharedSQLContext { } test("ZeroArgument non-deterministic UDF") { - spark.udf.register("random0", () => { Math.random() }, deterministic = false) + spark.udf.registerNonDeterministic("random0", () => { Math.random() }) val df = sql("SELECT random0()") assert(df.logicalPlan.asInstanceOf[Project].projectList.forall(!_.deterministic)) assert(df.head().getDouble(0) >= 0.0) From 96e9eb727f21ce6c7b62913dff4a3f62d187230e Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Tue, 18 Jul 2017 13:52:18 -0700 Subject: [PATCH 05/12] fix. --- .../src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java index 7e8c1c88232ee..bbe1427f75458 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java @@ -123,7 +123,7 @@ public void udf6Test() { Assert.assertEquals(1, result.getInt(0)); } - public static class randUDFTest implements UDF1 { + public static class RandUDFTest implements UDF1 { @Override public Double call(Integer i) { return i + Math.random(); @@ -133,7 +133,7 @@ public Double call(Integer i) { @SuppressWarnings("unchecked") @Test public void udf7Test() { - spark.udf().registerJava("randUDF", randUDFTest.class.getName(), DataTypes.DoubleType, false); + spark.udf().registerJava("randUDF", RandUDFTest.class.getName(), DataTypes.DoubleType, false); Row result = spark.sql("SELECT randUDF(1)").head(); Assert.assertTrue(result.getDouble(0) >= 0.0); } From d0a90865ca7c6a9afd6fbb28b3e8d1c9c602013c Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Wed, 19 Jul 2017 14:23:56 -0700 Subject: [PATCH 06/12] fix. --- .../apache/spark/sql/UDFRegistration.scala | 568 ++++-------------- .../sql/expressions/UserDefinedFunction.scala | 14 +- .../org/apache/spark/sql/functions.scala | 113 ++-- .../scala/org/apache/spark/sql/UDFSuite.scala | 7 +- 4 files changed, 217 insertions(+), 485 deletions(-) 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 07efdd0fec6ba..babd7800717d0 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 @@ -79,8 +79,15 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Register a user-defined function (UDF), for a UDF that's already defined using the DataFrame - * API (i.e. of type UserDefinedFunction). + * Registers a user-defined function (UDF), for a UDF that's already defined using the Dataset + * API (i.e. of type UserDefinedFunction). To change a UDF to nondeterministic, call the API + * [[UserDefinedFunction.asNondeterministic()]]. + * + * Example: + * {{{ + * val foo = udf(() => { Math.random() }) + * spark.udf.register("random", foo.asNondeterministic()) + * }}} * * @param name the name of the UDF. * @param udf the UDF needs to be registered. @@ -104,35 +111,22 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends val inputTypes = (1 to x).foldRight("Nil")((i, s) => {s"ScalaReflection.schemaFor[A$i].dataType :: $s"}) println(s""" /** - * Registers a Scala closure of ${x} arguments as user-defined function (UDF). + * Registers a deterministic Scala closure of ${x} arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ def register[$typeTags](name: String, func: Function$x[$types]): UserDefinedFunction = { - registerUDF(name, func, deterministic = true) - } - - /** - * Registers a non-deterministic Scala closure of ${x} arguments as user-defined function (UDF). - * @tparam RT return type of UDF. - * @since 2.3.0 - */ - def registerNonDeterministic[$typeTags](name: String, func: Function$x[$types]): UserDefinedFunction = { - registerUDF(name, func, deterministic = false) - } - - private def registerUDF[$typeTags](name: String, func: Function$x[$types], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputTypes = Try($inputTypes).toOption def builder(e: Seq[Expression]) = if (e.length == $x) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: $x; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - if (!deterministic) udf.nonDeterministic() else udf + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name) + if (nullable) udf else udf.asNonNullabe() }""") } @@ -145,18 +139,18 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends val funcCall = if (i == 0) "() => func" else "func" println(s""" |/** - | * Registers a user-defined function with ${i} arguments. + | * Registers a deterministic user-defined function with ${i} arguments. | * @since $version | */ |def register(name: String, f: UDF$i[$extTypeArgs], returnType: DataType): Unit = { - | registerNonDeterministic(name, f, returnType, deterministic = true) + | register(name, f, returnType, deterministic = true) |} | |/** | * Registers a user-defined function with ${i} arguments. | * @since 2.3.0 | */ - |def registerNonDeterministic(name: String, f: UDF$i[$extTypeArgs], returnType: DataType, deterministic: Boolean): Unit = { + |def register(name: String, f: UDF$i[$extTypeArgs], returnType: DataType, deterministic: Boolean): Unit = { | val func = f$anyCast.call($anyParams) | def builder(e: Seq[Expression]) = if (e.length == $i) { | ScalaUDF($funcCall, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) @@ -170,739 +164,440 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ /** - * Registers a Scala closure of 0 arguments as user-defined function (UDF). + * Registers a deterministic Scala closure of 0 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ def register[RT: TypeTag](name: String, func: Function0[RT]): UserDefinedFunction = { - registerUDF(name, func, deterministic = true) - } - - /** - * Registers a non-deterministic Scala closure of 0 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. - * @since 2.3.0 - */ - def registerNonDeterministic[RT: TypeTag](name: String, func: Function0[RT]): UserDefinedFunction = { - registerUDF(name, func, deterministic = false) - } - - private def registerUDF[RT: TypeTag](name: String, func: Function0[RT], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputTypes = Try(Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 0) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 0; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - if (!deterministic) udf.nonDeterministic() else udf + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name) + if (nullable) udf else udf.asNonNullabe() } /** - * Registers a Scala closure of 1 arguments as user-defined function (UDF). + * Registers a deterministic Scala closure of 1 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag](name: String, func: Function1[A1, RT]): UserDefinedFunction = { - registerUDF(name, func, deterministic = true) - } - - /** - * Registers a non-deterministic Scala closure of 1 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. - * @since 2.3.0 - */ - def registerNonDeterministic[RT: TypeTag, A1: TypeTag](name: String, func: Function1[A1, RT]): UserDefinedFunction = { - registerUDF(name, func, deterministic = false) - } - - private def registerUDF[RT: TypeTag, A1: TypeTag](name: String, func: Function1[A1, RT], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 1) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 1; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - if (!deterministic) udf.nonDeterministic() else udf + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name) + if (nullable) udf else udf.asNonNullabe() } /** - * Registers a Scala closure of 2 arguments as user-defined function (UDF). + * Registers a deterministic Scala closure of 2 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag](name: String, func: Function2[A1, A2, RT]): UserDefinedFunction = { - registerUDF(name, func, deterministic = true) - } - - /** - * Registers a non-deterministic Scala closure of 2 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. - * @since 2.3.0 - */ - def registerNonDeterministic[RT: TypeTag, A1: TypeTag, A2: TypeTag](name: String, func: Function2[A1, A2, RT]): UserDefinedFunction = { - registerUDF(name, func, deterministic = false) - } - - private def registerUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag](name: String, func: Function2[A1, A2, RT], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 2) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 2; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - if (!deterministic) udf.nonDeterministic() else udf + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name) + if (nullable) udf else udf.asNonNullabe() } /** - * Registers a Scala closure of 3 arguments as user-defined function (UDF). + * Registers a deterministic Scala closure of 3 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag](name: String, func: Function3[A1, A2, A3, RT]): UserDefinedFunction = { - registerUDF(name, func, deterministic = true) - } - - /** - * Registers a non-deterministic Scala closure of 3 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. - * @since 2.3.0 - */ - def registerNonDeterministic[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag](name: String, func: Function3[A1, A2, A3, RT]): UserDefinedFunction = { - registerUDF(name, func, deterministic = false) - } - - private def registerUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag](name: String, func: Function3[A1, A2, A3, RT], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 3) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 3; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - if (!deterministic) udf.nonDeterministic() else udf + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name) + if (nullable) udf else udf.asNonNullabe() } /** - * Registers a Scala closure of 4 arguments as user-defined function (UDF). + * Registers a deterministic Scala closure of 4 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag](name: String, func: Function4[A1, A2, A3, A4, RT]): UserDefinedFunction = { - registerUDF(name, func, deterministic = true) - } - - /** - * Registers a non-deterministic Scala closure of 4 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. - * @since 2.3.0 - */ - def registerNonDeterministic[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag](name: String, func: Function4[A1, A2, A3, A4, RT]): UserDefinedFunction = { - registerUDF(name, func, deterministic = false) - } - - private def registerUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag](name: String, func: Function4[A1, A2, A3, A4, RT], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 4) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 4; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - if (!deterministic) udf.nonDeterministic() else udf + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name) + if (nullable) udf else udf.asNonNullabe() } /** - * Registers a Scala closure of 5 arguments as user-defined function (UDF). + * Registers a deterministic Scala closure of 5 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ 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 = { - registerUDF(name, func, deterministic = true) - } - - /** - * Registers a non-deterministic Scala closure of 5 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. - * @since 2.3.0 - */ - def registerNonDeterministic[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag](name: String, func: Function5[A1, A2, A3, A4, A5, RT]): UserDefinedFunction = { - registerUDF(name, func, deterministic = false) - } - - private def registerUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag](name: String, func: Function5[A1, A2, A3, A4, A5, RT], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 5) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 5; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - if (!deterministic) udf.nonDeterministic() else udf + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name) + if (nullable) udf else udf.asNonNullabe() } /** - * Registers a Scala closure of 6 arguments as user-defined function (UDF). + * Registers a deterministic Scala closure of 6 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ 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 = { - registerUDF(name, func, deterministic = true) - } - - /** - * Registers a non-deterministic Scala closure of 6 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. - * @since 2.3.0 - */ - def registerNonDeterministic[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 = { - registerUDF(name, func, deterministic = false) - } - - private def registerUDF[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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 6) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 6; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - if (!deterministic) udf.nonDeterministic() else udf + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name) + if (nullable) udf else udf.asNonNullabe() } /** - * Registers a Scala closure of 7 arguments as user-defined function (UDF). + * Registers a deterministic Scala closure of 7 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ 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 = { - registerUDF(name, func, deterministic = true) - } - - /** - * Registers a non-deterministic Scala closure of 7 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. - * @since 2.3.0 - */ - def registerNonDeterministic[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 = { - registerUDF(name, func, deterministic = false) - } - - private def registerUDF[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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 7) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 7; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - if (!deterministic) udf.nonDeterministic() else udf + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name) + if (nullable) udf else udf.asNonNullabe() } /** - * Registers a Scala closure of 8 arguments as user-defined function (UDF). + * Registers a deterministic Scala closure of 8 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ 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 = { - registerUDF(name, func, deterministic = true) - } - - /** - * Registers a non-deterministic Scala closure of 8 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. - * @since 2.3.0 - */ - def registerNonDeterministic[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 = { - registerUDF(name, func, deterministic = false) - } - - private def registerUDF[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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 8) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 8; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - if (!deterministic) udf.nonDeterministic() else udf + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name) + if (nullable) udf else udf.asNonNullabe() } /** - * Registers a Scala closure of 9 arguments as user-defined function (UDF). + * Registers a deterministic Scala closure of 9 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ 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 = { - registerUDF(name, func, deterministic = true) - } - - /** - * Registers a non-deterministic Scala closure of 9 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. - * @since 2.3.0 - */ - def registerNonDeterministic[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 = { - registerUDF(name, func, deterministic = false) - } - - private def registerUDF[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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 9) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 9; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - if (!deterministic) udf.nonDeterministic() else udf + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name) + if (nullable) udf else udf.asNonNullabe() } /** - * Registers a Scala closure of 10 arguments as user-defined function (UDF). + * Registers a deterministic Scala closure of 10 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ 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 = { - registerUDF(name, func, deterministic = true) - } - - /** - * Registers a non-deterministic Scala closure of 10 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. - * @since 2.3.0 - */ - def registerNonDeterministic[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 = { - registerUDF(name, func, deterministic = false) - } - - private def registerUDF[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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 10) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 10; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - if (!deterministic) udf.nonDeterministic() else udf + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name) + if (nullable) udf else udf.asNonNullabe() } /** - * Registers a Scala closure of 11 arguments as user-defined function (UDF). + * Registers a deterministic Scala closure of 11 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ 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 = { - registerUDF(name, func, deterministic = true) - } - - /** - * Registers a non-deterministic Scala closure of 11 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. - * @since 2.3.0 - */ - def registerNonDeterministic[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 = { - registerUDF(name, func, deterministic = false) - } - - private def registerUDF[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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 11) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 11; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - if (!deterministic) udf.nonDeterministic() else udf + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name) + if (nullable) udf else udf.asNonNullabe() } /** - * Registers a Scala closure of 12 arguments as user-defined function (UDF). + * Registers a deterministic Scala closure of 12 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ 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 = { - registerUDF(name, func, deterministic = true) - } - - /** - * Registers a non-deterministic Scala closure of 12 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. - * @since 2.3.0 - */ - def registerNonDeterministic[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 = { - registerUDF(name, func, deterministic = false) - } - - private def registerUDF[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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 12) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 12; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - if (!deterministic) udf.nonDeterministic() else udf + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name) + if (nullable) udf else udf.asNonNullabe() } /** - * Registers a Scala closure of 13 arguments as user-defined function (UDF). + * Registers a deterministic Scala closure of 13 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ 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 = { - registerUDF(name, func, deterministic = true) - } - - /** - * Registers a non-deterministic Scala closure of 13 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. - * @since 2.3.0 - */ - def registerNonDeterministic[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 = { - registerUDF(name, func, deterministic = false) - } - - private def registerUDF[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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 13) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 13; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - if (!deterministic) udf.nonDeterministic() else udf + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name) + if (nullable) udf else udf.asNonNullabe() } /** - * Registers a Scala closure of 14 arguments as user-defined function (UDF). + * Registers a deterministic Scala closure of 14 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ 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 = { - registerUDF(name, func, deterministic = true) - } - - /** - * Registers a non-deterministic Scala closure of 14 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. - * @since 2.3.0 - */ - def registerNonDeterministic[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 = { - registerUDF(name, func, deterministic = false) - } - - private def registerUDF[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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 14) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 14; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - if (!deterministic) udf.nonDeterministic() else udf + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name) + if (nullable) udf else udf.asNonNullabe() } /** - * Registers a Scala closure of 15 arguments as user-defined function (UDF). + * Registers a deterministic Scala closure of 15 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ 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 = { - registerUDF(name, func, deterministic = true) - } - - /** - * Registers a non-deterministic Scala closure of 15 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. - * @since 2.3.0 - */ - def registerNonDeterministic[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 = { - registerUDF(name, func, deterministic = false) - } - - private def registerUDF[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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 15) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 15; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - if (!deterministic) udf.nonDeterministic() else udf + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name) + if (nullable) udf else udf.asNonNullabe() } /** - * Registers a Scala closure of 16 arguments as user-defined function (UDF). + * Registers a deterministic Scala closure of 16 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ 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 = { - registerUDF(name, func, deterministic = true) - } - - /** - * Registers a non-deterministic Scala closure of 16 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. - * @since 2.3.0 - */ - def registerNonDeterministic[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 = { - registerUDF(name, func, deterministic = false) - } - - private def registerUDF[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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 16) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 16; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - if (!deterministic) udf.nonDeterministic() else udf + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name) + if (nullable) udf else udf.asNonNullabe() } /** - * Registers a Scala closure of 17 arguments as user-defined function (UDF). + * Registers a deterministic Scala closure of 17 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ 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 = { - registerUDF(name, func, deterministic = true) - } - - /** - * Registers a non-deterministic Scala closure of 17 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. - * @since 2.3.0 - */ - def registerNonDeterministic[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 = { - registerUDF(name, func, deterministic = false) - } - - private def registerUDF[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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 17) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 17; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - if (!deterministic) udf.nonDeterministic() else udf + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name) + if (nullable) udf else udf.asNonNullabe() } /** - * Registers a Scala closure of 18 arguments as user-defined function (UDF). + * Registers a deterministic Scala closure of 18 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ 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 = { - registerUDF(name, func, deterministic = true) - } - - /** - * Registers a non-deterministic Scala closure of 18 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. - * @since 2.3.0 - */ - def registerNonDeterministic[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 = { - registerUDF(name, func, deterministic = false) - } - - private def registerUDF[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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 18) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 18; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - if (!deterministic) udf.nonDeterministic() else udf + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name) + if (nullable) udf else udf.asNonNullabe() } /** - * Registers a Scala closure of 19 arguments as user-defined function (UDF). + * Registers a deterministic Scala closure of 19 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ 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 = { - registerUDF(name, func, deterministic = true) - } - - /** - * Registers a non-deterministic Scala closure of 19 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. - * @since 2.3.0 - */ - def registerNonDeterministic[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 = { - registerUDF(name, func, deterministic = false) - } - - private def registerUDF[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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 19) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 19; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - if (!deterministic) udf.nonDeterministic() else udf + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name) + if (nullable) udf else udf.asNonNullabe() } /** - * Registers a Scala closure of 20 arguments as user-defined function (UDF). + * Registers a deterministic Scala closure of 20 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ 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 = { - registerUDF(name, func, deterministic = true) - } - - /** - * Registers a non-deterministic Scala closure of 20 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. - * @since 2.3.0 - */ - def registerNonDeterministic[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 = { - registerUDF(name, func, deterministic = false) - } - - private def registerUDF[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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 20) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 20; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - if (!deterministic) udf.nonDeterministic() else udf + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name) + if (nullable) udf else udf.asNonNullabe() } /** - * Registers a Scala closure of 21 arguments as user-defined function (UDF). + * Registers a deterministic Scala closure of 21 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ 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 = { - registerUDF(name, func, deterministic = true) - } - - /** - * Registers a non-deterministic Scala closure of 21 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. - * @since 2.3.0 - */ - def registerNonDeterministic[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 = { - registerUDF(name, func, deterministic = false) - } - - private def registerUDF[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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 21) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 21; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - if (!deterministic) udf.nonDeterministic() else udf + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name) + if (nullable) udf else udf.asNonNullabe() } /** - * Registers a Scala closure of 22 arguments as user-defined function (UDF). + * Registers a deterministic Scala closure of 22 arguments as user-defined function (UDF). * @tparam RT return type of UDF. * @since 1.3.0 */ 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 = { - registerUDF(name, func, deterministic = true) - } - - /** - * Registers a non-deterministic Scala closure of 22 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. - * @since 2.3.0 - */ - def registerNonDeterministic[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 = { - registerUDF(name, func, deterministic = false) - } - - private def registerUDF[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], deterministic: Boolean): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] 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 def builder(e: Seq[Expression]) = if (e.length == 22) { - ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic) + ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 22; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable) - if (!deterministic) udf.nonDeterministic() else udf + val udf = UserDefinedFunction(func, dataType, inputTypes).withName(name) + if (nullable) udf else udf.asNonNullabe() } ////////////////////////////////////////////////////////////////////////////////////////////// @@ -1012,9 +707,8 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } } - /** - * Registers a user-defined function with 0 arguments. + * Registers a deterministic user-defined function with 0 arguments. * @since 2.3.0 */ def register(name: String, f: UDF0[_], returnType: DataType): Unit = { @@ -1037,7 +731,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a user-defined function with 1 arguments. + * Registers a deterministic user-defined function with 1 arguments. * @since 1.3.0 */ def register(name: String, f: UDF1[_, _], returnType: DataType): Unit = { @@ -1060,7 +754,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a user-defined function with 2 arguments. + * Registers a deterministic user-defined function with 2 arguments. * @since 1.3.0 */ def register(name: String, f: UDF2[_, _, _], returnType: DataType): Unit = { @@ -1083,7 +777,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a user-defined function with 3 arguments. + * Registers a deterministic user-defined function with 3 arguments. * @since 1.3.0 */ def register(name: String, f: UDF3[_, _, _, _], returnType: DataType): Unit = { @@ -1106,7 +800,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a user-defined function with 4 arguments. + * Registers a deterministic user-defined function with 4 arguments. * @since 1.3.0 */ def register(name: String, f: UDF4[_, _, _, _, _], returnType: DataType): Unit = { @@ -1129,7 +823,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a user-defined function with 5 arguments. + * Registers a deterministic user-defined function with 5 arguments. * @since 1.3.0 */ def register(name: String, f: UDF5[_, _, _, _, _, _], returnType: DataType): Unit = { @@ -1152,7 +846,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a user-defined function with 6 arguments. + * Registers a deterministic user-defined function with 6 arguments. * @since 1.3.0 */ def register(name: String, f: UDF6[_, _, _, _, _, _, _], returnType: DataType): Unit = { @@ -1175,7 +869,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a user-defined function with 7 arguments. + * Registers a deterministic user-defined function with 7 arguments. * @since 1.3.0 */ def register(name: String, f: UDF7[_, _, _, _, _, _, _, _], returnType: DataType): Unit = { @@ -1198,7 +892,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a user-defined function with 8 arguments. + * Registers a deterministic user-defined function with 8 arguments. * @since 1.3.0 */ def register(name: String, f: UDF8[_, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { @@ -1221,7 +915,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a user-defined function with 9 arguments. + * Registers a deterministic user-defined function with 9 arguments. * @since 1.3.0 */ def register(name: String, f: UDF9[_, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { @@ -1244,7 +938,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a user-defined function with 10 arguments. + * Registers a deterministic user-defined function with 10 arguments. * @since 1.3.0 */ def register(name: String, f: UDF10[_, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { @@ -1267,7 +961,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a user-defined function with 11 arguments. + * Registers a deterministic user-defined function with 11 arguments. * @since 1.3.0 */ def register(name: String, f: UDF11[_, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { @@ -1290,7 +984,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a user-defined function with 12 arguments. + * Registers a deterministic user-defined function with 12 arguments. * @since 1.3.0 */ def register(name: String, f: UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { @@ -1313,7 +1007,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a user-defined function with 13 arguments. + * Registers a deterministic user-defined function with 13 arguments. * @since 1.3.0 */ def register(name: String, f: UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { @@ -1336,7 +1030,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a user-defined function with 14 arguments. + * Registers a deterministic user-defined function with 14 arguments. * @since 1.3.0 */ def register(name: String, f: UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { @@ -1359,7 +1053,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a user-defined function with 15 arguments. + * Registers a deterministic user-defined function with 15 arguments. * @since 1.3.0 */ def register(name: String, f: UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { @@ -1382,7 +1076,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a user-defined function with 16 arguments. + * Registers a deterministic user-defined function with 16 arguments. * @since 1.3.0 */ def register(name: String, f: UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { @@ -1405,7 +1099,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a user-defined function with 17 arguments. + * Registers a deterministic user-defined function with 17 arguments. * @since 1.3.0 */ def register(name: String, f: UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { @@ -1428,7 +1122,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a user-defined function with 18 arguments. + * Registers a deterministic user-defined function with 18 arguments. * @since 1.3.0 */ def register(name: String, f: UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { @@ -1451,7 +1145,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a user-defined function with 19 arguments. + * Registers a deterministic user-defined function with 19 arguments. * @since 1.3.0 */ def register(name: String, f: UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { @@ -1474,7 +1168,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a user-defined function with 20 arguments. + * Registers a deterministic user-defined function with 20 arguments. * @since 1.3.0 */ def register(name: String, f: UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { @@ -1497,7 +1191,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a user-defined function with 21 arguments. + * Registers a deterministic user-defined function with 21 arguments. * @since 1.3.0 */ def register(name: String, f: UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { @@ -1520,7 +1214,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a user-defined function with 22 arguments. + * Registers a deterministic user-defined function with 22 arguments. * @since 1.3.0 */ def register(name: String, f: UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala index 433c9ac8d0b8b..0e734b7514053 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.expressions import org.apache.spark.annotation.InterfaceStability import org.apache.spark.sql.catalyst.expressions.ScalaUDF import org.apache.spark.sql.Column -import org.apache.spark.sql.functions import org.apache.spark.sql.types.DataType /** @@ -55,7 +54,8 @@ case class UserDefinedFunction protected[sql] ( def nullable: Boolean = _nullable /** - * Returns true when the UDF is deterministic. + * Returns true iff the UDF is deterministic, i.e. the UDF produces the same output given the same + * input. * * @since 2.3.0 */ @@ -97,26 +97,26 @@ case class UserDefinedFunction protected[sql] ( } /** - * Updates UserDefinedFunction with a given nullability. + * Updates UserDefinedFunction to non-nullable. * * @since 2.3.0 */ - def withNullability(nullable: Boolean): UserDefinedFunction = { + def asNonNullabe(): UserDefinedFunction = { if (nullable == _nullable) { this } else { val udf = copyAll() - udf._nullable = nullable + udf._nullable = false udf } } /** - * Updates UserDefinedFunction to non-deterministic. + * Updates UserDefinedFunction to nondeterministic. * * @since 2.3.0 */ - def nonDeterministic(): UserDefinedFunction = { + def asNondeterministic(): UserDefinedFunction = { if (!_deterministic) { this } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index ebdeb42b0bfb1..344a450e9b729 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -3185,8 +3185,10 @@ object functions { val inputTypes = (1 to x).foldRight("Nil")((i, s) => {s"ScalaReflection.schemaFor(typeTag[A$i]).dataType :: $s"}) println(s""" /** - * Defines a user-defined function of ${x} arguments as user-defined function (UDF). - * The data types are automatically inferred based on the function's signature. + * Defines a deterministic user-defined function of ${x} arguments as user-defined + * function (UDF). The data types are automatically inferred based on the function's + * signature. To change a UDF to nondeterministic, call the API + * [[UserDefinedFunction.asNondeterministic()]]. * * @group udf_funcs * @since 1.3.0 @@ -3194,15 +3196,18 @@ object functions { def udf[$typeTags](f: Function$x[$types]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputTypes = Try($inputTypes).toOption - UserDefinedFunction(f, dataType, inputTypes).withNullability(nullable) + val udf = UserDefinedFunction(f, dataType, inputTypes) + if (nullable) udf else udf.asNonNullabe() }""") } */ /** - * Defines a user-defined function of 0 arguments as user-defined function (UDF). - * The data types are automatically inferred based on the function's signature. + * Defines a deterministic user-defined function of 0 arguments as user-defined + * function (UDF). The data types are automatically inferred based on the function's + * signature. To change a UDF to nondeterministic, call the API + * [[UserDefinedFunction.asNondeterministic()]]. * * @group udf_funcs * @since 1.3.0 @@ -3210,12 +3215,15 @@ object functions { def udf[RT: TypeTag](f: Function0[RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputTypes = Try(Nil).toOption - UserDefinedFunction(f, dataType, inputTypes).withNullability(nullable) + val udf = UserDefinedFunction(f, dataType, inputTypes) + if (nullable) udf else udf.asNonNullabe() } /** - * Defines a user-defined function of 1 arguments as user-defined function (UDF). - * The data types are automatically inferred based on the function's signature. + * Defines a deterministic user-defined function of 1 arguments as user-defined + * function (UDF). The data types are automatically inferred based on the function's + * signature. To change a UDF to nondeterministic, call the API + * [[UserDefinedFunction.asNondeterministic()]]. * * @group udf_funcs * @since 1.3.0 @@ -3223,12 +3231,15 @@ object functions { def udf[RT: TypeTag, A1: TypeTag](f: Function1[A1, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputTypes = Try(ScalaReflection.schemaFor(typeTag[A1]).dataType :: Nil).toOption - UserDefinedFunction(f, dataType, inputTypes).withNullability(nullable) + val udf = UserDefinedFunction(f, dataType, inputTypes) + if (nullable) udf else udf.asNonNullabe() } /** - * Defines a user-defined function of 2 arguments as user-defined function (UDF). - * The data types are automatically inferred based on the function's signature. + * Defines a deterministic user-defined function of 2 arguments as user-defined + * function (UDF). The data types are automatically inferred based on the function's + * signature. To change a UDF to nondeterministic, call the API + * [[UserDefinedFunction.asNondeterministic()]]. * * @group udf_funcs * @since 1.3.0 @@ -3236,12 +3247,15 @@ object functions { def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag](f: Function2[A1, A2, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputTypes = Try(ScalaReflection.schemaFor(typeTag[A1]).dataType :: ScalaReflection.schemaFor(typeTag[A2]).dataType :: Nil).toOption - UserDefinedFunction(f, dataType, inputTypes).withNullability(nullable) + val udf = UserDefinedFunction(f, dataType, inputTypes) + if (nullable) udf else udf.asNonNullabe() } /** - * Defines a user-defined function of 3 arguments as user-defined function (UDF). - * The data types are automatically inferred based on the function's signature. + * Defines a deterministic user-defined function of 3 arguments as user-defined + * function (UDF). The data types are automatically inferred based on the function's + * signature. To change a UDF to nondeterministic, call the API + * [[UserDefinedFunction.asNondeterministic()]]. * * @group udf_funcs * @since 1.3.0 @@ -3249,12 +3263,15 @@ object functions { def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag](f: Function3[A1, A2, A3, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputTypes = Try(ScalaReflection.schemaFor(typeTag[A1]).dataType :: ScalaReflection.schemaFor(typeTag[A2]).dataType :: ScalaReflection.schemaFor(typeTag[A3]).dataType :: Nil).toOption - UserDefinedFunction(f, dataType, inputTypes).withNullability(nullable) + val udf = UserDefinedFunction(f, dataType, inputTypes) + if (nullable) udf else udf.asNonNullabe() } /** - * Defines a user-defined function of 4 arguments as user-defined function (UDF). - * The data types are automatically inferred based on the function's signature. + * Defines a deterministic user-defined function of 4 arguments as user-defined + * function (UDF). The data types are automatically inferred based on the function's + * signature. To change a UDF to nondeterministic, call the API + * [[UserDefinedFunction.asNondeterministic()]]. * * @group udf_funcs * @since 1.3.0 @@ -3262,12 +3279,15 @@ object functions { def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag](f: Function4[A1, A2, A3, A4, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputTypes = Try(ScalaReflection.schemaFor(typeTag[A1]).dataType :: ScalaReflection.schemaFor(typeTag[A2]).dataType :: ScalaReflection.schemaFor(typeTag[A3]).dataType :: ScalaReflection.schemaFor(typeTag[A4]).dataType :: Nil).toOption - UserDefinedFunction(f, dataType, inputTypes).withNullability(nullable) + val udf = UserDefinedFunction(f, dataType, inputTypes) + if (nullable) udf else udf.asNonNullabe() } /** - * Defines a user-defined function of 5 arguments as user-defined function (UDF). - * The data types are automatically inferred based on the function's signature. + * Defines a deterministic user-defined function of 5 arguments as user-defined + * function (UDF). The data types are automatically inferred based on the function's + * signature. To change a UDF to nondeterministic, call the API + * [[UserDefinedFunction.asNondeterministic()]]. * * @group udf_funcs * @since 1.3.0 @@ -3275,12 +3295,15 @@ object functions { def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag](f: Function5[A1, A2, A3, A4, A5, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputTypes = Try(ScalaReflection.schemaFor(typeTag[A1]).dataType :: ScalaReflection.schemaFor(typeTag[A2]).dataType :: ScalaReflection.schemaFor(typeTag[A3]).dataType :: ScalaReflection.schemaFor(typeTag[A4]).dataType :: ScalaReflection.schemaFor(typeTag[A5]).dataType :: Nil).toOption - UserDefinedFunction(f, dataType, inputTypes).withNullability(nullable) + val udf = UserDefinedFunction(f, dataType, inputTypes) + if (nullable) udf else udf.asNonNullabe() } /** - * Defines a user-defined function of 6 arguments as user-defined function (UDF). - * The data types are automatically inferred based on the function's signature. + * Defines a deterministic user-defined function of 6 arguments as user-defined + * function (UDF). The data types are automatically inferred based on the function's + * signature. To change a UDF to nondeterministic, call the API + * [[UserDefinedFunction.asNondeterministic()]]. * * @group udf_funcs * @since 1.3.0 @@ -3288,12 +3311,15 @@ object functions { def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag](f: Function6[A1, A2, A3, A4, A5, A6, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputTypes = Try(ScalaReflection.schemaFor(typeTag[A1]).dataType :: ScalaReflection.schemaFor(typeTag[A2]).dataType :: ScalaReflection.schemaFor(typeTag[A3]).dataType :: ScalaReflection.schemaFor(typeTag[A4]).dataType :: ScalaReflection.schemaFor(typeTag[A5]).dataType :: ScalaReflection.schemaFor(typeTag[A6]).dataType :: Nil).toOption - UserDefinedFunction(f, dataType, inputTypes).withNullability(nullable) + val udf = UserDefinedFunction(f, dataType, inputTypes) + if (nullable) udf else udf.asNonNullabe() } /** - * Defines a user-defined function of 7 arguments as user-defined function (UDF). - * The data types are automatically inferred based on the function's signature. + * Defines a deterministic user-defined function of 7 arguments as user-defined + * function (UDF). The data types are automatically inferred based on the function's + * signature. To change a UDF to nondeterministic, call the API + * [[UserDefinedFunction.asNondeterministic()]]. * * @group udf_funcs * @since 1.3.0 @@ -3301,12 +3327,15 @@ object functions { def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag](f: Function7[A1, A2, A3, A4, A5, A6, A7, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputTypes = Try(ScalaReflection.schemaFor(typeTag[A1]).dataType :: ScalaReflection.schemaFor(typeTag[A2]).dataType :: ScalaReflection.schemaFor(typeTag[A3]).dataType :: ScalaReflection.schemaFor(typeTag[A4]).dataType :: ScalaReflection.schemaFor(typeTag[A5]).dataType :: ScalaReflection.schemaFor(typeTag[A6]).dataType :: ScalaReflection.schemaFor(typeTag[A7]).dataType :: Nil).toOption - UserDefinedFunction(f, dataType, inputTypes).withNullability(nullable) + val udf = UserDefinedFunction(f, dataType, inputTypes) + if (nullable) udf else udf.asNonNullabe() } /** - * Defines a user-defined function of 8 arguments as user-defined function (UDF). - * The data types are automatically inferred based on the function's signature. + * Defines a deterministic user-defined function of 8 arguments as user-defined + * function (UDF). The data types are automatically inferred based on the function's + * signature. To change a UDF to nondeterministic, call the API + * [[UserDefinedFunction.asNondeterministic()]]. * * @group udf_funcs * @since 1.3.0 @@ -3314,12 +3343,15 @@ object functions { def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag](f: Function8[A1, A2, A3, A4, A5, A6, A7, A8, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputTypes = Try(ScalaReflection.schemaFor(typeTag[A1]).dataType :: ScalaReflection.schemaFor(typeTag[A2]).dataType :: ScalaReflection.schemaFor(typeTag[A3]).dataType :: ScalaReflection.schemaFor(typeTag[A4]).dataType :: ScalaReflection.schemaFor(typeTag[A5]).dataType :: ScalaReflection.schemaFor(typeTag[A6]).dataType :: ScalaReflection.schemaFor(typeTag[A7]).dataType :: ScalaReflection.schemaFor(typeTag[A8]).dataType :: Nil).toOption - UserDefinedFunction(f, dataType, inputTypes).withNullability(nullable) + val udf = UserDefinedFunction(f, dataType, inputTypes) + if (nullable) udf else udf.asNonNullabe() } /** - * Defines a user-defined function of 9 arguments as user-defined function (UDF). - * The data types are automatically inferred based on the function's signature. + * Defines a deterministic user-defined function of 9 arguments as user-defined + * function (UDF). The data types are automatically inferred based on the function's + * signature. To change a UDF to nondeterministic, call the API + * [[UserDefinedFunction.asNondeterministic()]]. * * @group udf_funcs * @since 1.3.0 @@ -3327,12 +3359,15 @@ object functions { def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag](f: Function9[A1, A2, A3, A4, A5, A6, A7, A8, A9, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputTypes = Try(ScalaReflection.schemaFor(typeTag[A1]).dataType :: ScalaReflection.schemaFor(typeTag[A2]).dataType :: ScalaReflection.schemaFor(typeTag[A3]).dataType :: ScalaReflection.schemaFor(typeTag[A4]).dataType :: ScalaReflection.schemaFor(typeTag[A5]).dataType :: ScalaReflection.schemaFor(typeTag[A6]).dataType :: ScalaReflection.schemaFor(typeTag[A7]).dataType :: ScalaReflection.schemaFor(typeTag[A8]).dataType :: ScalaReflection.schemaFor(typeTag[A9]).dataType :: Nil).toOption - UserDefinedFunction(f, dataType, inputTypes).withNullability(nullable) + val udf = UserDefinedFunction(f, dataType, inputTypes) + if (nullable) udf else udf.asNonNullabe() } /** - * Defines a user-defined function of 10 arguments as user-defined function (UDF). - * The data types are automatically inferred based on the function's signature. + * Defines a deterministic user-defined function of 10 arguments as user-defined + * function (UDF). The data types are automatically inferred based on the function's + * signature. To change a UDF to nondeterministic, call the API + * [[UserDefinedFunction.asNondeterministic()]]. * * @group udf_funcs * @since 1.3.0 @@ -3340,15 +3375,17 @@ object functions { def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag](f: Function10[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] val inputTypes = Try(ScalaReflection.schemaFor(typeTag[A1]).dataType :: ScalaReflection.schemaFor(typeTag[A2]).dataType :: ScalaReflection.schemaFor(typeTag[A3]).dataType :: ScalaReflection.schemaFor(typeTag[A4]).dataType :: ScalaReflection.schemaFor(typeTag[A5]).dataType :: ScalaReflection.schemaFor(typeTag[A6]).dataType :: ScalaReflection.schemaFor(typeTag[A7]).dataType :: ScalaReflection.schemaFor(typeTag[A8]).dataType :: ScalaReflection.schemaFor(typeTag[A9]).dataType :: ScalaReflection.schemaFor(typeTag[A10]).dataType :: Nil).toOption - UserDefinedFunction(f, dataType, inputTypes).withNullability(nullable) + val udf = UserDefinedFunction(f, dataType, inputTypes) + if (nullable) udf else udf.asNonNullabe() } // scalastyle:on parameter.number // scalastyle:on line.size.limit /** - * Defines a user-defined function (UDF) using a Scala closure. For this variant, the caller must - * specify the output data type, and there is no automatic input type coercion. + * Defines a deterministic user-defined function (UDF) using a Scala closure. For this variant, + * the caller must specify the output data type, and there is no automatic input type coercion. + * To change a UDF to nondeterministic, call the API [[UserDefinedFunction.asNondeterministic()]]. * * @param f A closure in Scala * @param dataType The output data type of the UDF 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 be4400ca191f1..bf688704836dc 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 @@ -113,17 +113,18 @@ class UDFSuite extends QueryTest with SharedSQLContext { } test("ZeroArgument non-deterministic UDF") { - spark.udf.registerNonDeterministic("random0", () => { Math.random() }) + val foo = udf(() => { Math.random() }) + spark.udf.register("random0", foo.asNondeterministic()) val df = sql("SELECT random0()") assert(df.logicalPlan.asInstanceOf[Project].projectList.forall(!_.deterministic)) assert(df.head().getDouble(0) >= 0.0) - val foo1 = udf(() => { Math.random() }).nonDeterministic() + val foo1 = udf(() => { Math.random() }).asNondeterministic() val df1 = testData.select(foo1()) assert(df1.logicalPlan.asInstanceOf[Project].projectList.forall(!_.deterministic)) assert(df1.head().getDouble(0) >= 0.0) - val foo2 = udf(() => { Math.random() }, DataTypes.DoubleType).nonDeterministic() + val foo2 = udf(() => { Math.random() }, DataTypes.DoubleType).asNondeterministic() val df2 = testData.select(foo2()) assert(df2.logicalPlan.asInstanceOf[Project].projectList.forall(!_.deterministic)) assert(df2.head().getDouble(0) >= 0.0) From 43bb9a9254d0d694b2be57ec6a3574d53e9c3141 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Wed, 19 Jul 2017 14:50:43 -0700 Subject: [PATCH 07/12] fix. --- .../src/main/scala/org/apache/spark/sql/UDFRegistration.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 babd7800717d0..99494d4e67817 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 @@ -81,7 +81,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends /** * Registers a user-defined function (UDF), for a UDF that's already defined using the Dataset * API (i.e. of type UserDefinedFunction). To change a UDF to nondeterministic, call the API - * [[UserDefinedFunction.asNondeterministic()]]. + * `UserDefinedFunction.asNondeterministic()`. * * Example: * {{{ From 0ea4691d3ea979b86cb7c44f8290ff7dc805a8a7 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Wed, 19 Jul 2017 14:52:40 -0700 Subject: [PATCH 08/12] fix. --- .../org/apache/spark/sql/functions.scala | 26 +++++++++---------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 344a450e9b729..ccff00e570dbd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -3188,7 +3188,7 @@ object functions { * Defines a deterministic user-defined function of ${x} arguments as user-defined * function (UDF). The data types are automatically inferred based on the function's * signature. To change a UDF to nondeterministic, call the API - * [[UserDefinedFunction.asNondeterministic()]]. + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 1.3.0 @@ -3207,7 +3207,7 @@ object functions { * Defines a deterministic user-defined function of 0 arguments as user-defined * function (UDF). The data types are automatically inferred based on the function's * signature. To change a UDF to nondeterministic, call the API - * [[UserDefinedFunction.asNondeterministic()]]. + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 1.3.0 @@ -3223,7 +3223,7 @@ object functions { * Defines a deterministic user-defined function of 1 arguments as user-defined * function (UDF). The data types are automatically inferred based on the function's * signature. To change a UDF to nondeterministic, call the API - * [[UserDefinedFunction.asNondeterministic()]]. + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 1.3.0 @@ -3239,7 +3239,7 @@ object functions { * Defines a deterministic user-defined function of 2 arguments as user-defined * function (UDF). The data types are automatically inferred based on the function's * signature. To change a UDF to nondeterministic, call the API - * [[UserDefinedFunction.asNondeterministic()]]. + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 1.3.0 @@ -3255,7 +3255,7 @@ object functions { * Defines a deterministic user-defined function of 3 arguments as user-defined * function (UDF). The data types are automatically inferred based on the function's * signature. To change a UDF to nondeterministic, call the API - * [[UserDefinedFunction.asNondeterministic()]]. + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 1.3.0 @@ -3271,7 +3271,7 @@ object functions { * Defines a deterministic user-defined function of 4 arguments as user-defined * function (UDF). The data types are automatically inferred based on the function's * signature. To change a UDF to nondeterministic, call the API - * [[UserDefinedFunction.asNondeterministic()]]. + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 1.3.0 @@ -3287,7 +3287,7 @@ object functions { * Defines a deterministic user-defined function of 5 arguments as user-defined * function (UDF). The data types are automatically inferred based on the function's * signature. To change a UDF to nondeterministic, call the API - * [[UserDefinedFunction.asNondeterministic()]]. + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 1.3.0 @@ -3303,7 +3303,7 @@ object functions { * Defines a deterministic user-defined function of 6 arguments as user-defined * function (UDF). The data types are automatically inferred based on the function's * signature. To change a UDF to nondeterministic, call the API - * [[UserDefinedFunction.asNondeterministic()]]. + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 1.3.0 @@ -3319,7 +3319,7 @@ object functions { * Defines a deterministic user-defined function of 7 arguments as user-defined * function (UDF). The data types are automatically inferred based on the function's * signature. To change a UDF to nondeterministic, call the API - * [[UserDefinedFunction.asNondeterministic()]]. + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 1.3.0 @@ -3335,7 +3335,7 @@ object functions { * Defines a deterministic user-defined function of 8 arguments as user-defined * function (UDF). The data types are automatically inferred based on the function's * signature. To change a UDF to nondeterministic, call the API - * [[UserDefinedFunction.asNondeterministic()]]. + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 1.3.0 @@ -3351,7 +3351,7 @@ object functions { * Defines a deterministic user-defined function of 9 arguments as user-defined * function (UDF). The data types are automatically inferred based on the function's * signature. To change a UDF to nondeterministic, call the API - * [[UserDefinedFunction.asNondeterministic()]]. + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 1.3.0 @@ -3367,7 +3367,7 @@ object functions { * Defines a deterministic user-defined function of 10 arguments as user-defined * function (UDF). The data types are automatically inferred based on the function's * signature. To change a UDF to nondeterministic, call the API - * [[UserDefinedFunction.asNondeterministic()]]. + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 1.3.0 @@ -3385,7 +3385,7 @@ object functions { /** * Defines a deterministic user-defined function (UDF) using a Scala closure. For this variant, * the caller must specify the output data type, and there is no automatic input type coercion. - * To change a UDF to nondeterministic, call the API [[UserDefinedFunction.asNondeterministic()]]. + * To change a UDF to nondeterministic, call the API `UserDefinedFunction.asNondeterministic()`. * * @param f A closure in Scala * @param dataType The output data type of the UDF From 8422c4237133447c3c1f3cec077fbfe27767ac18 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Thu, 20 Jul 2017 15:01:50 -0700 Subject: [PATCH 09/12] fix. --- .../spark/sql/expressions/UserDefinedFunction.scala | 2 +- .../java/test/org/apache/spark/sql/JavaUDFSuite.java | 9 +-------- 2 files changed, 2 insertions(+), 9 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala index 0e734b7514053..97b921a622636 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala @@ -102,7 +102,7 @@ case class UserDefinedFunction protected[sql] ( * @since 2.3.0 */ def asNonNullabe(): UserDefinedFunction = { - if (nullable == _nullable) { + if (!nullable) { this } else { val udf = copyAll() diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java index bbe1427f75458..2521007dcda2c 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java @@ -123,17 +123,10 @@ public void udf6Test() { Assert.assertEquals(1, result.getInt(0)); } - public static class RandUDFTest implements UDF1 { - @Override - public Double call(Integer i) { - return i + Math.random(); - } - } - @SuppressWarnings("unchecked") @Test public void udf7Test() { - spark.udf().registerJava("randUDF", RandUDFTest.class.getName(), DataTypes.DoubleType, false); + spark.udf().registerJava("randUDF", JavaRandUDF.class.getName(), DataTypes.DoubleType, false); Row result = spark.sql("SELECT randUDF(1)").head(); Assert.assertTrue(result.getDouble(0) >= 0.0); } From 1b3aa22e07821b2303f3750470a5617b296ec317 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 25 Jul 2017 22:35:39 +0800 Subject: [PATCH 10/12] revert java UDF changes (#2) --- python/pyspark/sql/context.py | 24 +- .../apache/spark/sql/UDFRegistration.scala | 388 +++++------------- .../org/apache/spark/sql/JavaRandUDF.java | 30 -- .../org/apache/spark/sql/JavaUDFSuite.java | 19 - .../scala/org/apache/spark/sql/UDFSuite.scala | 9 +- 5 files changed, 104 insertions(+), 366 deletions(-) delete mode 100644 sql/core/src/test/java/test/org/apache/spark/sql/JavaRandUDF.java diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 19f10e98e0526..c44ab247fd3d3 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -28,7 +28,7 @@ from pyspark.sql.dataframe import DataFrame from pyspark.sql.readwriter import DataFrameReader from pyspark.sql.streaming import DataStreamReader -from pyspark.sql.types import DoubleType, IntegerType, Row, StringType +from pyspark.sql.types import IntegerType, Row, StringType from pyspark.sql.utils import install_exception_handler __all__ = ["SQLContext", "HiveContext", "UDFRegistration"] @@ -208,37 +208,29 @@ def registerFunction(self, name, f, returnType=StringType()): @ignore_unicode_prefix @since(2.1) - def registerJavaFunction(self, name, javaClassName, returnType=None, deterministic=True): + def registerJavaFunction(self, name, javaClassName, returnType=None): """Register a java UDF so it can be used in SQL statements. In addition to a name and the function itself, the return type can be optionally specified. When the return type is not specified we would infer it via reflection. - - :param name: name of the UDF. - :param javaClassName: fully qualified name of java class. - :param returnType: a :class:`pyspark.sql.types.DataType` object. - :param deterministic: a flag indicating if the UDF is deterministic. Deterministic UDF - returns same result each time it is invoked with a particular input. + :param name: name of the UDF + :param javaClassName: fully qualified name of java class + :param returnType: a :class:`pyspark.sql.types.DataType` object >>> sqlContext.registerJavaFunction("javaStringLength", ... "test.org.apache.spark.sql.JavaStringLength", IntegerType()) >>> sqlContext.sql("SELECT javaStringLength('test')").collect() - [Row(UDF:javaStringLength(test)=4)] + [Row(UDF(test)=4)] >>> sqlContext.registerJavaFunction("javaStringLength2", ... "test.org.apache.spark.sql.JavaStringLength") >>> sqlContext.sql("SELECT javaStringLength2('test')").collect() - [Row(UDF:javaStringLength2(test)=4)] - >>> sqlContext.registerJavaFunction("javaRand", - ... "test.org.apache.spark.sql.JavaRandUDF", DoubleType(), deterministic=False) - >>> sqlContext.sql("SELECT javaRand(3)").collect() # doctest: +SKIP - [Row(UDF:javaRand(3)=3.12345)] + [Row(UDF(test)=4)] """ jdt = None if returnType is not None: jdt = self.sparkSession._jsparkSession.parseDataType(returnType.json()) - self.sparkSession._jsparkSession.udf().registerJava( - name, javaClassName, jdt, deterministic) + self.sparkSession._jsparkSession.udf().registerJava(name, javaClassName, jdt) @ignore_unicode_prefix @since(2.3) 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 99494d4e67817..52dba6ebbb2ca 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 @@ -64,7 +64,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Register a user-defined aggregate function (UDAF). + * Registers a user-defined aggregate function (UDAF). * * @param name the name of the UDAF. * @param udaf the UDAF needs to be registered. @@ -81,12 +81,16 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends /** * Registers a user-defined function (UDF), for a UDF that's already defined using the Dataset * API (i.e. of type UserDefinedFunction). To change a UDF to nondeterministic, call the API - * `UserDefinedFunction.asNondeterministic()`. + * `UserDefinedFunction.asNondeterministic()`. To change a UDF to nonNullable, call the API + * `UserDefinedFunction.asNonNullabe()`. * * Example: * {{{ - * val foo = udf(() => { Math.random() }) + * val foo = udf(() => Math.random()) * spark.udf.register("random", foo.asNondeterministic()) + * + * val bar = udf(() => "bar") + * spark.udf.register("stringLit", bar.asNonNullabe()) * }}} * * @param name the name of the UDF. @@ -138,28 +142,20 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends val version = if (i == 0) "2.3.0" else "1.3.0" val funcCall = if (i == 0) "() => func" else "func" println(s""" - |/** - | * Registers a deterministic user-defined function with ${i} arguments. - | * @since $version - | */ - |def register(name: String, f: UDF$i[$extTypeArgs], returnType: DataType): Unit = { - | register(name, f, returnType, deterministic = true) - |} - | - |/** - | * Registers a user-defined function with ${i} arguments. - | * @since 2.3.0 - | */ - |def register(name: String, f: UDF$i[$extTypeArgs], returnType: DataType, deterministic: Boolean): Unit = { - | val func = f$anyCast.call($anyParams) - | def builder(e: Seq[Expression]) = if (e.length == $i) { - | ScalaUDF($funcCall, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) - | } else { - | throw new AnalysisException("Invalid number of arguments for function " + name + - | ". Expected: $i; Found: " + e.length) - | } - | functionRegistry.createOrReplaceTempFunction(name, builder) - |}""".stripMargin) + |/** + | * Register a user-defined function with ${i} arguments. + | * @since $version + | */ + |def register(name: String, f: UDF$i[$extTypeArgs], returnType: DataType): Unit = { + | val func = f$anyCast.call($anyParams) + | def builder(e: Seq[Expression]) = if (e.length == $i) { + | ScalaUDF($funcCall, returnType, e) + | } else { + | throw new AnalysisException("Invalid number of arguments for function " + name + + | ". Expected: $i; Found: " + e.length) + | } + | functionRegistry.createOrReplaceTempFunction(name, builder) + |}""".stripMargin) } */ @@ -604,7 +600,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends ////////////////////////////////////////////////////////////////////////////////////////////// /** - * Registers a Java UDF class using reflection, for use from pyspark + * Register a Java UDF class using reflection, for use from pyspark * * @param name udf name * @param className fully qualified class name of udf @@ -612,23 +608,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * via reflection. */ private[sql] def registerJava(name: String, className: String, returnDataType: DataType): Unit = { - registerJava(name, className, returnDataType, deterministic = true) - } - /** - * Registers a Java UDF class using reflection, for use from pyspark - * - * @param name udf name - * @param className fully qualified class name of udf - * @param returnDataType return type of udf. If it is null, spark would try to infer - * via reflection. - * @param deterministic True if the UDF is deterministic. Deterministic UDF returns same result - * each time it is invoked with a particular input. - */ - private[sql] def registerJava( - name: String, - className: String, - returnDataType: DataType, - deterministic: Boolean): Unit = { + try { val clazz = Utils.classForName(className) val udfInterfaces = clazz.getGenericInterfaces @@ -649,29 +629,29 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } udfInterfaces(0).getActualTypeArguments.length match { - case 1 => register(name, udf.asInstanceOf[UDF0[_]], returnType, deterministic) - case 2 => register(name, udf.asInstanceOf[UDF1[_, _]], returnType, deterministic) - case 3 => register(name, udf.asInstanceOf[UDF2[_, _, _]], returnType, deterministic) - case 4 => register(name, udf.asInstanceOf[UDF3[_, _, _, _]], returnType, deterministic) - case 5 => register(name, udf.asInstanceOf[UDF4[_, _, _, _, _]], returnType, deterministic) - case 6 => register(name, udf.asInstanceOf[UDF5[_, _, _, _, _, _]], returnType, deterministic) - case 7 => register(name, udf.asInstanceOf[UDF6[_, _, _, _, _, _, _]], returnType, deterministic) - case 8 => register(name, udf.asInstanceOf[UDF7[_, _, _, _, _, _, _, _]], returnType, deterministic) - case 9 => register(name, udf.asInstanceOf[UDF8[_, _, _, _, _, _, _, _, _]], returnType, deterministic) - case 10 => register(name, udf.asInstanceOf[UDF9[_, _, _, _, _, _, _, _, _, _]], returnType, deterministic) - case 11 => register(name, udf.asInstanceOf[UDF10[_, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic) - case 12 => register(name, udf.asInstanceOf[UDF11[_, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic) - case 13 => register(name, udf.asInstanceOf[UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic) - case 14 => register(name, udf.asInstanceOf[UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic) - case 15 => register(name, udf.asInstanceOf[UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic) - case 16 => register(name, udf.asInstanceOf[UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic) - case 17 => register(name, udf.asInstanceOf[UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic) - case 18 => register(name, udf.asInstanceOf[UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic) - case 19 => register(name, udf.asInstanceOf[UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic) - case 20 => register(name, udf.asInstanceOf[UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic) - case 21 => register(name, udf.asInstanceOf[UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic) - case 22 => register(name, udf.asInstanceOf[UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic) - case 23 => register(name, udf.asInstanceOf[UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType, deterministic) + case 1 => register(name, udf.asInstanceOf[UDF0[_]], returnType) + case 2 => register(name, udf.asInstanceOf[UDF1[_, _]], returnType) + case 3 => register(name, udf.asInstanceOf[UDF2[_, _, _]], returnType) + case 4 => register(name, udf.asInstanceOf[UDF3[_, _, _, _]], returnType) + case 5 => register(name, udf.asInstanceOf[UDF4[_, _, _, _, _]], returnType) + case 6 => register(name, udf.asInstanceOf[UDF5[_, _, _, _, _, _]], returnType) + case 7 => register(name, udf.asInstanceOf[UDF6[_, _, _, _, _, _, _]], returnType) + case 8 => register(name, udf.asInstanceOf[UDF7[_, _, _, _, _, _, _, _]], returnType) + case 9 => register(name, udf.asInstanceOf[UDF8[_, _, _, _, _, _, _, _, _]], returnType) + case 10 => register(name, udf.asInstanceOf[UDF9[_, _, _, _, _, _, _, _, _, _]], returnType) + case 11 => register(name, udf.asInstanceOf[UDF10[_, _, _, _, _, _, _, _, _, _, _]], returnType) + case 12 => register(name, udf.asInstanceOf[UDF11[_, _, _, _, _, _, _, _, _, _, _, _]], returnType) + case 13 => register(name, udf.asInstanceOf[UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) + case 14 => register(name, udf.asInstanceOf[UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) + case 15 => register(name, udf.asInstanceOf[UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) + case 16 => register(name, udf.asInstanceOf[UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) + case 17 => register(name, udf.asInstanceOf[UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) + case 18 => register(name, udf.asInstanceOf[UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) + case 19 => register(name, udf.asInstanceOf[UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) + case 20 => register(name, udf.asInstanceOf[UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) + case 21 => register(name, udf.asInstanceOf[UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) + case 22 => register(name, udf.asInstanceOf[UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) + case 23 => register(name, udf.asInstanceOf[UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) case n => throw new AnalysisException(s"UDF class with $n type arguments is not supported.") } @@ -708,21 +688,13 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a deterministic user-defined function with 0 arguments. + * Register a user-defined function with 0 arguments. * @since 2.3.0 */ def register(name: String, f: UDF0[_], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true) - } - - /** - * Registers a user-defined function with 0 arguments. - * @since 2.3.0 - */ - def register(name: String, f: UDF0[_], returnType: DataType, deterministic: Boolean): Unit = { val func = f.asInstanceOf[UDF0[Any]].call() def builder(e: Seq[Expression]) = if (e.length == 0) { - ScalaUDF(() => func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) + ScalaUDF(() => func, returnType, e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 0; Found: " + e.length) @@ -731,21 +703,13 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a deterministic user-defined function with 1 arguments. + * Register a user-defined function with 1 arguments. * @since 1.3.0 */ def register(name: String, f: UDF1[_, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true) - } - - /** - * Registers a user-defined function with 1 arguments. - * @since 2.3.0 - */ - def register(name: String, f: UDF1[_, _], returnType: DataType, deterministic: Boolean): Unit = { val func = f.asInstanceOf[UDF1[Any, Any]].call(_: Any) def builder(e: Seq[Expression]) = if (e.length == 1) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) + ScalaUDF(func, returnType, e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 1; Found: " + e.length) @@ -754,21 +718,13 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a deterministic user-defined function with 2 arguments. + * Register a user-defined function with 2 arguments. * @since 1.3.0 */ def register(name: String, f: UDF2[_, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true) - } - - /** - * Registers a user-defined function with 2 arguments. - * @since 2.3.0 - */ - def register(name: String, f: UDF2[_, _, _], returnType: DataType, deterministic: Boolean): Unit = { val func = f.asInstanceOf[UDF2[Any, Any, Any]].call(_: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 2) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) + ScalaUDF(func, returnType, e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 2; Found: " + e.length) @@ -777,21 +733,13 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a deterministic user-defined function with 3 arguments. + * Register a user-defined function with 3 arguments. * @since 1.3.0 */ def register(name: String, f: UDF3[_, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true) - } - - /** - * Registers a user-defined function with 3 arguments. - * @since 2.3.0 - */ - def register(name: String, f: UDF3[_, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { val func = f.asInstanceOf[UDF3[Any, Any, Any, Any]].call(_: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 3) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) + ScalaUDF(func, returnType, e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 3; Found: " + e.length) @@ -800,21 +748,13 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a deterministic user-defined function with 4 arguments. + * Register a user-defined function with 4 arguments. * @since 1.3.0 */ def register(name: String, f: UDF4[_, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true) - } - - /** - * Registers a user-defined function with 4 arguments. - * @since 2.3.0 - */ - def register(name: String, f: UDF4[_, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { val func = f.asInstanceOf[UDF4[Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 4) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) + ScalaUDF(func, returnType, e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 4; Found: " + e.length) @@ -823,21 +763,13 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a deterministic user-defined function with 5 arguments. + * Register a user-defined function with 5 arguments. * @since 1.3.0 */ def register(name: String, f: UDF5[_, _, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true) - } - - /** - * Registers a user-defined function with 5 arguments. - * @since 2.3.0 - */ - def register(name: String, f: UDF5[_, _, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { val func = f.asInstanceOf[UDF5[Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 5) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) + ScalaUDF(func, returnType, e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 5; Found: " + e.length) @@ -846,21 +778,13 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a deterministic user-defined function with 6 arguments. + * Register a user-defined function with 6 arguments. * @since 1.3.0 */ def register(name: String, f: UDF6[_, _, _, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true) - } - - /** - * Registers a user-defined function with 6 arguments. - * @since 2.3.0 - */ - def register(name: String, f: UDF6[_, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { val func = f.asInstanceOf[UDF6[Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 6) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) + ScalaUDF(func, returnType, e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 6; Found: " + e.length) @@ -869,21 +793,13 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a deterministic user-defined function with 7 arguments. + * Register a user-defined function with 7 arguments. * @since 1.3.0 */ def register(name: String, f: UDF7[_, _, _, _, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true) - } - - /** - * Registers a user-defined function with 7 arguments. - * @since 2.3.0 - */ - def register(name: String, f: UDF7[_, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { val func = f.asInstanceOf[UDF7[Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 7) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) + ScalaUDF(func, returnType, e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 7; Found: " + e.length) @@ -892,21 +808,13 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a deterministic user-defined function with 8 arguments. + * Register a user-defined function with 8 arguments. * @since 1.3.0 */ def register(name: String, f: UDF8[_, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true) - } - - /** - * Registers a user-defined function with 8 arguments. - * @since 2.3.0 - */ - def register(name: String, f: UDF8[_, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { val func = f.asInstanceOf[UDF8[Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 8) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) + ScalaUDF(func, returnType, e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 8; Found: " + e.length) @@ -915,21 +823,13 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a deterministic user-defined function with 9 arguments. + * Register a user-defined function with 9 arguments. * @since 1.3.0 */ def register(name: String, f: UDF9[_, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true) - } - - /** - * Registers a user-defined function with 9 arguments. - * @since 2.3.0 - */ - def register(name: String, f: UDF9[_, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 9) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) + ScalaUDF(func, returnType, e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 9; Found: " + e.length) @@ -938,21 +838,13 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a deterministic user-defined function with 10 arguments. + * Register a user-defined function with 10 arguments. * @since 1.3.0 */ def register(name: String, f: UDF10[_, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true) - } - - /** - * Registers a user-defined function with 10 arguments. - * @since 2.3.0 - */ - def register(name: String, f: UDF10[_, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 10) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) + ScalaUDF(func, returnType, e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 10; Found: " + e.length) @@ -961,21 +853,13 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a deterministic user-defined function with 11 arguments. + * Register a user-defined function with 11 arguments. * @since 1.3.0 */ def register(name: String, f: UDF11[_, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true) - } - - /** - * Registers a user-defined function with 11 arguments. - * @since 2.3.0 - */ - def register(name: String, f: UDF11[_, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 11) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) + ScalaUDF(func, returnType, e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 11; Found: " + e.length) @@ -984,21 +868,13 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a deterministic user-defined function with 12 arguments. + * Register a user-defined function with 12 arguments. * @since 1.3.0 */ def register(name: String, f: UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true) - } - - /** - * Registers a user-defined function with 12 arguments. - * @since 2.3.0 - */ - def register(name: String, f: UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 12) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) + ScalaUDF(func, returnType, e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 12; Found: " + e.length) @@ -1007,21 +883,13 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a deterministic user-defined function with 13 arguments. + * Register a user-defined function with 13 arguments. * @since 1.3.0 */ def register(name: String, f: UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true) - } - - /** - * Registers a user-defined function with 13 arguments. - * @since 2.3.0 - */ - def register(name: String, f: UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 13) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) + ScalaUDF(func, returnType, e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 13; Found: " + e.length) @@ -1030,21 +898,13 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a deterministic user-defined function with 14 arguments. + * Register a user-defined function with 14 arguments. * @since 1.3.0 */ def register(name: String, f: UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true) - } - - /** - * Registers a user-defined function with 14 arguments. - * @since 2.3.0 - */ - def register(name: String, f: UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 14) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) + ScalaUDF(func, returnType, e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 14; Found: " + e.length) @@ -1053,21 +913,13 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a deterministic user-defined function with 15 arguments. + * Register a user-defined function with 15 arguments. * @since 1.3.0 */ def register(name: String, f: UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true) - } - - /** - * Registers a user-defined function with 15 arguments. - * @since 2.3.0 - */ - def register(name: String, f: UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 15) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) + ScalaUDF(func, returnType, e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 15; Found: " + e.length) @@ -1076,21 +928,13 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a deterministic user-defined function with 16 arguments. + * Register a user-defined function with 16 arguments. * @since 1.3.0 */ def register(name: String, f: UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true) - } - - /** - * Registers a user-defined function with 16 arguments. - * @since 2.3.0 - */ - def register(name: String, f: UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 16) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) + ScalaUDF(func, returnType, e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 16; Found: " + e.length) @@ -1099,21 +943,13 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a deterministic user-defined function with 17 arguments. + * Register a user-defined function with 17 arguments. * @since 1.3.0 */ def register(name: String, f: UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true) - } - - /** - * Registers a user-defined function with 17 arguments. - * @since 2.3.0 - */ - def register(name: String, f: UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 17) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) + ScalaUDF(func, returnType, e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 17; Found: " + e.length) @@ -1122,21 +958,13 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a deterministic user-defined function with 18 arguments. + * Register a user-defined function with 18 arguments. * @since 1.3.0 */ def register(name: String, f: UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true) - } - - /** - * Registers a user-defined function with 18 arguments. - * @since 2.3.0 - */ - def register(name: String, f: UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 18) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) + ScalaUDF(func, returnType, e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 18; Found: " + e.length) @@ -1145,21 +973,13 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a deterministic user-defined function with 19 arguments. + * Register a user-defined function with 19 arguments. * @since 1.3.0 */ def register(name: String, f: UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true) - } - - /** - * Registers a user-defined function with 19 arguments. - * @since 2.3.0 - */ - def register(name: String, f: UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 19) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) + ScalaUDF(func, returnType, e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 19; Found: " + e.length) @@ -1168,21 +988,13 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a deterministic user-defined function with 20 arguments. + * Register a user-defined function with 20 arguments. * @since 1.3.0 */ def register(name: String, f: UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true) - } - - /** - * Registers a user-defined function with 20 arguments. - * @since 2.3.0 - */ - def register(name: String, f: UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 20) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) + ScalaUDF(func, returnType, e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 20; Found: " + e.length) @@ -1191,21 +1003,13 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a deterministic user-defined function with 21 arguments. + * Register a user-defined function with 21 arguments. * @since 1.3.0 */ def register(name: String, f: UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true) - } - - /** - * Registers a user-defined function with 21 arguments. - * @since 2.3.0 - */ - def register(name: String, f: UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 21) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) + ScalaUDF(func, returnType, e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 21; Found: " + e.length) @@ -1214,21 +1018,13 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends } /** - * Registers a deterministic user-defined function with 22 arguments. + * Register a user-defined function with 22 arguments. * @since 1.3.0 */ def register(name: String, f: UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { - register(name, f, returnType, deterministic = true) - } - - /** - * Registers a user-defined function with 22 arguments. - * @since 2.3.0 - */ - def register(name: String, f: UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, deterministic: Boolean): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 22) { - ScalaUDF(func, returnType, e, inputTypes = Nil, udfName = Some(name), nullable = true, deterministic) + ScalaUDF(func, returnType, e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 22; Found: " + e.length) diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaRandUDF.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaRandUDF.java deleted file mode 100644 index df806bc18288d..0000000000000 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaRandUDF.java +++ /dev/null @@ -1,30 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package test.org.apache.spark.sql; - -import org.apache.spark.sql.api.java.UDF1; - -/** - * It is used for register Java UDF from PySpark - */ -public class JavaRandUDF implements UDF1 { - @Override - public Double call(Integer i) { - return i + Math.random(); - } -} diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java index 2521007dcda2c..5bf1888826186 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java @@ -28,7 +28,6 @@ import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.api.java.UDF1; import org.apache.spark.sql.api.java.UDF2; import org.apache.spark.sql.types.DataTypes; @@ -122,22 +121,4 @@ public void udf6Test() { Row result = spark.sql("SELECT returnOne()").head(); Assert.assertEquals(1, result.getInt(0)); } - - @SuppressWarnings("unchecked") - @Test - public void udf7Test() { - spark.udf().registerJava("randUDF", JavaRandUDF.class.getName(), DataTypes.DoubleType, false); - Row result = spark.sql("SELECT randUDF(1)").head(); - Assert.assertTrue(result.getDouble(0) >= 0.0); - } - - @SuppressWarnings("unchecked") - @Test - public void udf8Test() { - spark.udf().register("randUDF", (Integer i) -> i + Math.random(), DataTypes.DoubleType, false); - Row result = spark.sql("SELECT randUDF(1)").head(); - Assert.assertTrue(result.getDouble(0) >= 0.0); - Assert.assertTrue("EXPLAIN outputs are expected to contain the UDF name.", - spark.sql("EXPLAIN SELECT randUDF(1) AS r").collectAsList().toString().contains("randUDF")); - } } 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 bf688704836dc..52f3fe2b18662 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 @@ -113,19 +113,18 @@ class UDFSuite extends QueryTest with SharedSQLContext { } test("ZeroArgument non-deterministic UDF") { - val foo = udf(() => { Math.random() }) + val foo = udf(() => Math.random()) spark.udf.register("random0", foo.asNondeterministic()) val df = sql("SELECT random0()") assert(df.logicalPlan.asInstanceOf[Project].projectList.forall(!_.deterministic)) assert(df.head().getDouble(0) >= 0.0) - val foo1 = udf(() => { Math.random() }).asNondeterministic() - val df1 = testData.select(foo1()) + val df1 = testData.select(foo()) assert(df1.logicalPlan.asInstanceOf[Project].projectList.forall(!_.deterministic)) assert(df1.head().getDouble(0) >= 0.0) - val foo2 = udf(() => { Math.random() }, DataTypes.DoubleType).asNondeterministic() - val df2 = testData.select(foo2()) + val bar = udf(() => Math.random(), DataTypes.DoubleType).asNondeterministic() + val df2 = testData.select(bar()) assert(df2.logicalPlan.asInstanceOf[Project].projectList.forall(!_.deterministic)) assert(df2.head().getDouble(0) >= 0.0) } From bf060d66f62bcb54b7177af24a2dd7a9198b9864 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 25 Jul 2017 22:48:04 +0800 Subject: [PATCH 11/12] fix java udf name (#3) --- python/pyspark/sql/context.py | 4 +- .../apache/spark/sql/UDFRegistration.scala | 48 +++++++++---------- 2 files changed, 26 insertions(+), 26 deletions(-) diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index c44ab247fd3d3..b1e723cdecef3 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -220,11 +220,11 @@ def registerJavaFunction(self, name, javaClassName, returnType=None): >>> sqlContext.registerJavaFunction("javaStringLength", ... "test.org.apache.spark.sql.JavaStringLength", IntegerType()) >>> sqlContext.sql("SELECT javaStringLength('test')").collect() - [Row(UDF(test)=4)] + [Row(UDF:javaStringLength(test)=4)] >>> sqlContext.registerJavaFunction("javaStringLength2", ... "test.org.apache.spark.sql.JavaStringLength") >>> sqlContext.sql("SELECT javaStringLength2('test')").collect() - [Row(UDF(test)=4)] + [Row(UDF:javaStringLength2(test)=4)] """ jdt = None 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 52dba6ebbb2ca..737afb4ac564e 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 @@ -149,7 +149,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends |def register(name: String, f: UDF$i[$extTypeArgs], returnType: DataType): Unit = { | val func = f$anyCast.call($anyParams) | def builder(e: Seq[Expression]) = if (e.length == $i) { - | ScalaUDF($funcCall, returnType, e) + | ScalaUDF($funcCall, returnType, e, udfName = Some(name)) | } else { | throw new AnalysisException("Invalid number of arguments for function " + name + | ". Expected: $i; Found: " + e.length) @@ -694,7 +694,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF0[_], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF0[Any]].call() def builder(e: Seq[Expression]) = if (e.length == 0) { - ScalaUDF(() => func, returnType, e) + ScalaUDF(() => func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 0; Found: " + e.length) @@ -709,7 +709,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF1[_, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF1[Any, Any]].call(_: Any) def builder(e: Seq[Expression]) = if (e.length == 1) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 1; Found: " + e.length) @@ -724,7 +724,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF2[_, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF2[Any, Any, Any]].call(_: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 2) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 2; Found: " + e.length) @@ -739,7 +739,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF3[_, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF3[Any, Any, Any, Any]].call(_: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 3) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 3; Found: " + e.length) @@ -754,7 +754,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF4[_, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF4[Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 4) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 4; Found: " + e.length) @@ -769,7 +769,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF5[_, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF5[Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 5) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 5; Found: " + e.length) @@ -784,7 +784,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF6[_, _, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF6[Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 6) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 6; Found: " + e.length) @@ -799,7 +799,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF7[_, _, _, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF7[Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 7) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 7; Found: " + e.length) @@ -814,7 +814,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF8[_, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF8[Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 8) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 8; Found: " + e.length) @@ -829,7 +829,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF9[_, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 9) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 9; Found: " + e.length) @@ -844,7 +844,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF10[_, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 10) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 10; Found: " + e.length) @@ -859,7 +859,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF11[_, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 11) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 11; Found: " + e.length) @@ -874,7 +874,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 12) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 12; Found: " + e.length) @@ -889,7 +889,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 13) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 13; Found: " + e.length) @@ -904,7 +904,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 14) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 14; Found: " + e.length) @@ -919,7 +919,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 15) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 15; Found: " + e.length) @@ -934,7 +934,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 16) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 16; Found: " + e.length) @@ -949,7 +949,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 17) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 17; Found: " + e.length) @@ -964,7 +964,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 18) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 18; Found: " + e.length) @@ -979,7 +979,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 19) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 19; Found: " + e.length) @@ -994,7 +994,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 20) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 20; Found: " + e.length) @@ -1009,7 +1009,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 21) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 21; Found: " + e.length) @@ -1024,7 +1024,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { 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) def builder(e: Seq[Expression]) = if (e.length == 22) { - ScalaUDF(func, returnType, e) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 22; Found: " + e.length) From a54010a34bdbfd0aa7c94be4ec8359896231a3e3 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Tue, 25 Jul 2017 13:09:29 -0700 Subject: [PATCH 12/12] fix test case. --- sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 52f3fe2b18662..7f1c009ca6e7a 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 @@ -119,7 +119,8 @@ class UDFSuite extends QueryTest with SharedSQLContext { assert(df.logicalPlan.asInstanceOf[Project].projectList.forall(!_.deterministic)) assert(df.head().getDouble(0) >= 0.0) - val df1 = testData.select(foo()) + val foo1 = foo.asNondeterministic() + val df1 = testData.select(foo1()) assert(df1.logicalPlan.asInstanceOf[Project].projectList.forall(!_.deterministic)) assert(df1.head().getDouble(0) >= 0.0)