You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by gatorsmile <gi...@git.apache.org> on 2017/05/03 22:34:40 UTC

[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic and distinc...

GitHub user gatorsmile opened a pull request:

    https://github.com/apache/spark/pull/17848

    [SPARK-20586] [SQL] Add deterministic and distinctLike to ScalaUDF and JavaUDF [WIP]

    ### What changes were proposed in this pull request?
    https://hive.apache.org/javadocs/r2.0.1/api/org/apache/hadoop/hive/ql/udf/UDFType.html
    
    Like Hive UDFType, we should allow users to add the extra flags for ScalaUDF and JavaUDF too. {{stateful}}/{{impliesOrder}} are not applicable to ScalaUDF. Thus, we only add the following two flags. 
    
    - deterministic: Certain optimizations should not be applied if UDF is not deterministic. Deterministic UDF returns same result each time it is invoked with a particular input. This determinism just needs to hold within the context of a query.
    
    - distinctLike: A UDF is considered distinctLike if the UDF can be evaluated on just the distinct values of a column. Examples include min and max UDFs. This information is used by metadata-only optimizer.
    
    When the deterministic flag is not correctly set, the results could be wrong. 
    
    Also corrected an issue in the ScalaUDF name loss in UDF registration. 
    
    For ScalaUDF in Dataset APIs, users can call the following three extra APIs for `UserDefinedFunction` to make the corresponding changes.
    - `withName`: Updates UserDefinedFunction with a given name.
    - `nonDeterministic`: Updates UserDefinedFunction to non-deterministic.
    - `distinctLike` : Updates UserDefinedFunction to distinctLike.
    
    ### How was this patch tested?
    Added test cases for both ScalaUDF and JavaUDF 

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/gatorsmile/spark udfRegister

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/17848.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #17848
    
----
commit 7df4d9d7fc59011ebefb470f29288d43d51ebaee
Author: Xiao Li <ga...@gmail.com>
Date:   2017-04-19T22:11:42Z

    temp fix1

commit 88fde5f8a80496faf1474622e8bbbd2969a8231f
Author: Xiao Li <ga...@gmail.com>
Date:   2017-05-03T22:26:04Z

    fix.

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79594/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r128161850
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala ---
    @@ -103,4 +110,19 @@ case class UserDefinedFunction protected[sql] (
           udf
         }
       }
    +
    +  /**
    +   * Updates UserDefinedFunction to non-deterministic.
    +   *
    +   * @since 2.3.0
    +   */
    +  def nonDeterministic(): UserDefinedFunction = {
    --- End diff --
    
    this is not mutable, we do copy. Maybe `withNondeterministic` is better?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF and Ja...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic and distinc...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r116373739
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala ---
    @@ -144,11 +181,24 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends
         val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: Nil).toOption
         def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name))
         functionRegistry.registerFunction(name, builder)
    -    UserDefinedFunction(func, dataType, inputTypes)
    +    UserDefinedFunction(func, dataType, inputTypes, Some(name))
       }
     
       /**
    -   * Register a Scala closure of 2 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 2.3.0
    +   */
    +  def register[RT: TypeTag, A1: TypeTag](name: String, func: Function1[A1, RT], deterministic : Boolean, distinctLike: Boolean): UserDefinedFunction = {
    --- End diff --
    
    After discussion with others, we decide to use the way you proposed. The related two PRs have been merged. Will make a change in this PR too. Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    cc @cloud-fan @sameeragarwal 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/17848


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    **[Test build #79942 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79942/testReport)** for PR 17848 at commit [`a54010a`](https://github.com/apache/spark/commit/a54010a34bdbfd0aa7c94be4ec8359896231a3e3).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF and Ja...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic and distinc...

Posted by maropu <gi...@git.apache.org>.
Github user maropu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r116891635
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala ---
    @@ -115,6 +115,22 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends
               def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable)
               functionRegistry.registerFunction(name, builder)
               UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable)
    --- End diff --
    
    I probably miss your point though, I suggested code below;
    ```
      /**
       * 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]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable, deterministic, distinctLike)
        functionRegistry.registerFunction(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 your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF and Ja...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    **[Test build #79727 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79727/testReport)** for PR 17848 at commit [`96e9eb7`](https://github.com/apache/spark/commit/96e9eb727f21ce6c7b62913dff4a3f62d187230e).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic and distinc...

Posted by maropu <gi...@git.apache.org>.
Github user maropu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r115251151
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala ---
    @@ -42,17 +42,81 @@ import org.apache.spark.sql.types.DataType
      * @since 1.3.0
      */
     @InterfaceStability.Stable
    -case class UserDefinedFunction protected[sql] (
    -    f: AnyRef,
    -    dataType: DataType,
    -    inputTypes: Option[Seq[DataType]]) {
    +class UserDefinedFunction protected[sql] (
    +    val f: AnyRef,
    +    val dataType: DataType,
    +    val inputTypes: Option[Seq[DataType]],
    +    val name: Option[String] = None,
    +    val deterministic: Boolean = true,
    +    val distinctLike: Boolean = false) {
    --- End diff --
    
    Since this breaks bin. compatibility (I know you are familiar with the point), the target of this pr is `spark-v3.0.0`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r127866406
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala ---
    @@ -103,4 +110,19 @@ case class UserDefinedFunction protected[sql] (
           udf
         }
       }
    +
    +  /**
    +   * Updates UserDefinedFunction to non-deterministic.
    +   *
    +   * @since 2.3.0
    +   */
    +  def nonDeterministic(): UserDefinedFunction = {
    --- End diff --
    
    not a big deal, let's keep it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    **[Test build #76986 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/76986/testReport)** for PR 17848 at commit [`f738e9c`](https://github.com/apache/spark/commit/f738e9c39b7bc12d9b8c0b72316222687f033e1d).
     * This patch **fails Python style tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r128657173
  
    --- Diff: 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()`.
    --- End diff --
    
    Sure. Will do



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r128626143
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala ---
    @@ -130,460 +138,507 @@ 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 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 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       //////////////////////////////////////////////////////////////////////////////////////////////
       //////////////////////////////////////////////////////////////////////////////////////////////
     
       /**
    -   * 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
        * @param returnDataType  return type of udf. If it is null, spark would try to infer
        *                        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(
    --- End diff --
    
    Actually, our JAVA API directly uses it. To JAVA APIs, they are not private at all.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic and distinc...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r116879496
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala ---
    @@ -85,8 +94,9 @@ case class UserDefinedFunction protected[sql] (
        * @since 2.3.0
        */
       def withName(name: String): this.type = {
    -    this._nameOption = Option(name)
    -    this
    +    val udf = copyAll()
    +    udf._nameOption = Option(name)
    --- End diff --
    
    cc @maropu 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic and distinc...

Posted by maropu <gi...@git.apache.org>.
Github user maropu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r116893118
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala ---
    @@ -85,8 +94,9 @@ case class UserDefinedFunction protected[sql] (
        * @since 2.3.0
        */
       def withName(name: String): this.type = {
    -    this._nameOption = Option(name)
    -    this
    +    val udf = copyAll()
    +    udf._nameOption = Option(name)
    --- End diff --
    
    I know your intention here (you probably mean we should not update values even in `var` variables) though, is it okay that the code below has four times object allocation in the worst case? I'm a bit worried about this point;
    
    ```
    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
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF and Ja...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    **[Test build #79780 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79780/testReport)** for PR 17848 at commit [`0ea4691`](https://github.com/apache/spark/commit/0ea4691d3ea979b86cb7c44f8290ff7dc805a8a7).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r128630372
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/functions.scala ---
    @@ -3185,170 +3185,207 @@ 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
          */
         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
        */
       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
        */
       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
        */
       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
        */
       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
        */
       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
        */
       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
        */
       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
        */
       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
        */
       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
        */
       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
        */
       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,
    --- End diff --
    
    Unfortunately nope, although we accept `AnyRef`. 
    ```
    java.lang.ClassCastException: java.lang.Class cannot be cast to scala.Function1
    
    	at org.apache.spark.sql.catalyst.expressions.ScalaUDF.<init>(ScalaUDF.scala:92)
    	at org.apache.spark.sql.expressions.UserDefinedFunction.apply(UserDefinedFunction.scala:70)
    	at org.apache.spark.sql.UDFRegistration.org$apache$spark$sql$UDFRegistration$$builder$2(UDFRegistration.scala:99)
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    **[Test build #76428 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/76428/testReport)** for PR 17848 at commit [`88fde5f`](https://github.com/apache/spark/commit/88fde5f8a80496faf1474622e8bbbd2969a8231f).
     * This patch **fails MiMa tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF and Ja...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79780/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r128668538
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/functions.scala ---
    @@ -3185,170 +3185,207 @@ 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
          */
         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
        */
       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
        */
       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
        */
       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
        */
       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
        */
       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
        */
       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
        */
       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
        */
       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
        */
       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
        */
       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
        */
       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,
    --- End diff --
    
    damn...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    **[Test build #79594 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79594/testReport)** for PR 17848 at commit [`eb9a7fc`](https://github.com/apache/spark/commit/eb9a7fc28b0f9c15f39178aebfecaacece1e4523).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF and Ja...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    **[Test build #79658 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79658/testReport)** for PR 17848 at commit [`a336ffc`](https://github.com/apache/spark/commit/a336ffc1bd70050d82eef79f6179e25231b23aa1).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    **[Test build #79228 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79228/testReport)** for PR 17848 at commit [`0c65322`](https://github.com/apache/spark/commit/0c65322f27cf9e1f5d686a45135798d1eb21a964).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r128657154
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala ---
    @@ -130,460 +138,507 @@ 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 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 = {
    --- End diff --
    
    So far, the impl of `def udf(f: AnyRef, dataType: DataType)` does not support Java UDF



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r128444145
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala ---
    @@ -130,460 +138,507 @@ 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 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 = {
    --- End diff --
    
    do we need this? I think for java UDF we can also build a `UserDefiendFunction` first and call `asNondeterminstic` or `asNonNullable`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79228/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79130/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    **[Test build #76986 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/76986/testReport)** for PR 17848 at commit [`f738e9c`](https://github.com/apache/spark/commit/f738e9c39b7bc12d9b8c0b72316222687f033e1d).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    **[Test build #76915 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/76915/testReport)** for PR 17848 at commit [`00b4dff`](https://github.com/apache/spark/commit/00b4dff4e4b57f1406d99957655e2cb3bd85ad8e).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `        throw new IOException(s\"UDF class $className doesn't implement any UDF interface\")`
      * `        throw new IOException(s\"It is invalid to implement multiple UDF interfaces, UDF class $className\")`
      * `            case n => logError(s\"UDF class with $n type arguments is not supported \")`
      * `            logError(s\"Can not instantiate class $className, please make sure it has public non argument constructor\")`
      * `      case e: ClassNotFoundException => logError(s\"Can not load class $className, please make sure it is on the classpath\")`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic and distinc...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r114669314
  
    --- Diff: 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 _)
    --- End diff --
    
    It sounds like Scala compiler is not smart enough... 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF and Ja...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/76919/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r128439889
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala ---
    @@ -84,22 +90,38 @@ 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
       }
     
       /**
    -   * 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) {
    --- End diff --
    
    nit: `if (!nullable)`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic and distinc...

Posted by maropu <gi...@git.apache.org>.
Github user maropu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r125388937
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala ---
    @@ -85,8 +94,9 @@ case class UserDefinedFunction protected[sql] (
        * @since 2.3.0
        */
       def withName(name: String): this.type = {
    -    this._nameOption = Option(name)
    -    this
    +    val udf = copyAll()
    +    udf._nameOption = Option(name)
    --- End diff --
    
    yea, I know. I just meant we added an interface `newInstance(name, nullable, determinism)` there.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/76428/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    **[Test build #76922 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/76922/testReport)** for PR 17848 at commit [`d276b44`](https://github.com/apache/spark/commit/d276b44ce3f68344ae1151c930105fe291a925ec).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    **[Test build #76919 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/76919/testReport)** for PR 17848 at commit [`387af4b`](https://github.com/apache/spark/commit/387af4b98b3b32a89904d05678eb58d76852160c).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r127754207
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala ---
    @@ -103,4 +110,19 @@ case class UserDefinedFunction protected[sql] (
           udf
         }
       }
    +
    +  /**
    +   * Updates UserDefinedFunction to non-deterministic.
    +   *
    +   * @since 2.3.0
    +   */
    +  def nonDeterministic(): UserDefinedFunction = {
    --- End diff --
    
    Originally, this is from @rxin. I am fine for both ways. If needed, we can have both?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r128575854
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala ---
    @@ -130,460 +138,507 @@ 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 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 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       //////////////////////////////////////////////////////////////////////////////////////////////
       //////////////////////////////////////////////////////////////////////////////////////////////
     
       /**
    -   * 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
        * @param returnDataType  return type of udf. If it is null, spark would try to infer
        *                        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(
    --- End diff --
    
    This is for PySpark. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    **[Test build #76428 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/76428/testReport)** for PR 17848 at commit [`88fde5f`](https://github.com/apache/spark/commit/88fde5f8a80496faf1474622e8bbbd2969a8231f).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    **[Test build #79130 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79130/testReport)** for PR 17848 at commit [`0aa6475`](https://github.com/apache/spark/commit/0aa64755009701c1d37de27c48926b4f46373fa8).
     * This patch **fails Python style tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic and distinc...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r114669131
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala ---
    @@ -93,9 +96,22 @@ class UDFSuite extends QueryTest with SharedSQLContext {
         assert(sql("SELECT strLenScala('test')").head().getInt(0) === 4)
       }
     
    -  test("ZeroArgument UDF") {
    -    spark.udf.register("random0", () => { Math.random()})
    --- End diff --
    
    Even our test case does not follow our assumption. We do not expect users to define non-deterministic UDF before this PR.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF and Ja...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    **[Test build #79779 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79779/testReport)** for PR 17848 at commit [`43bb9a9`](https://github.com/apache/spark/commit/43bb9a9254d0d694b2be57ec6a3574d53e9c3141).
     * This patch **fails to generate documentation**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF and Ja...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79776/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    **[Test build #79936 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79936/testReport)** for PR 17848 at commit [`bf060d6`](https://github.com/apache/spark/commit/bf060d66f62bcb54b7177af24a2dd7a9198b9864).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org

[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF and Ja...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    **[Test build #79935 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79935/testReport)** for PR 17848 at commit [`1b3aa22`](https://github.com/apache/spark/commit/1b3aa22e07821b2303f3750470a5617b296ec317).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org

[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    **[Test build #76918 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/76918/testReport)** for PR 17848 at commit [`c496b62`](https://github.com/apache/spark/commit/c496b6219e58fcd6d223eb2579087a76ce911310).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/76922/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r128444945
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/functions.scala ---
    @@ -3185,170 +3185,207 @@ 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
          */
         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
        */
       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
        */
       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
        */
       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
        */
       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
        */
       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
        */
       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
        */
       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
        */
       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
        */
       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
        */
       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
        */
       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,
    --- End diff --
    
    not only scala closure, I think java UDF class is also supported here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r128576017
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala ---
    @@ -130,460 +138,507 @@ 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 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 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       //////////////////////////////////////////////////////////////////////////////////////////////
       //////////////////////////////////////////////////////////////////////////////////////////////
     
       /**
    -   * 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
        * @param returnDataType  return type of udf. If it is null, spark would try to infer
        *                        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(
    --- End diff --
    
    uh... We can feel free to make the change on the interface. : )


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r128444432
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala ---
    @@ -130,460 +138,507 @@ 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 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 = {
    --- End diff --
    
    BTW although we don't have `def udf(f: UDF1[_, _]): UserDefinedFunction` APIs, we do have `def udf(f: AnyRef, dataType: DataType): UserDefinedFunction` which can be used for java udf.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Sorry for a late update. Taking care of two kids alone is really a challenging task. Will update the PR now.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    **[Test build #79935 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79935/testReport)** for PR 17848 at commit [`1b3aa22`](https://github.com/apache/spark/commit/1b3aa22e07821b2303f3750470a5617b296ec317).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org

[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r127866355
  
    --- Diff: 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 _)
    --- End diff --
    
    this sounds like a source code compatibility issue, can we look into it?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r128668866
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala ---
    @@ -130,460 +138,507 @@ 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 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 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       //////////////////////////////////////////////////////////////////////////////////////////////
       //////////////////////////////////////////////////////////////////////////////////////////////
     
       /**
    -   * 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
        * @param returnDataType  return type of udf. If it is null, spark would try to infer
        *                        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(
    --- End diff --
    
    BTW can we use default parameter? will it break java compatibility?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    **[Test build #76918 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/76918/testReport)** for PR 17848 at commit [`c496b62`](https://github.com/apache/spark/commit/c496b6219e58fcd6d223eb2579087a76ce911310).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    **[Test build #76915 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/76915/testReport)** for PR 17848 at commit [`00b4dff`](https://github.com/apache/spark/commit/00b4dff4e4b57f1406d99957655e2cb3bd85ad8e).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    **[Test build #79942 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79942/testReport)** for PR 17848 at commit [`a54010a`](https://github.com/apache/spark/commit/a54010a34bdbfd0aa7c94be4ec8359896231a3e3).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    **[Test build #79130 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79130/testReport)** for PR 17848 at commit [`0aa6475`](https://github.com/apache/spark/commit/0aa64755009701c1d37de27c48926b4f46373fa8).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF and Ja...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    **[Test build #79727 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79727/testReport)** for PR 17848 at commit [`96e9eb7`](https://github.com/apache/spark/commit/96e9eb727f21ce6c7b62913dff4a3f62d187230e).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `  public static class RandUDFTest implements UDF1<Integer, Double> `


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF and Ja...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    **[Test build #79810 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79810/testReport)** for PR 17848 at commit [`8422c42`](https://github.com/apache/spark/commit/8422c4237133447c3c1f3cec077fbfe27767ac18).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    **[Test build #79594 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79594/testReport)** for PR 17848 at commit [`eb9a7fc`](https://github.com/apache/spark/commit/eb9a7fc28b0f9c15f39178aebfecaacece1e4523).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r127635707
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala ---
    @@ -103,4 +110,19 @@ case class UserDefinedFunction protected[sql] (
           udf
         }
       }
    +
    +  /**
    +   * Updates UserDefinedFunction to non-deterministic.
    +   *
    +   * @since 2.3.0
    +   */
    +  def nonDeterministic(): UserDefinedFunction = {
    --- End diff --
    
    shall we follow existing naming style and make it `def withDeterministic(deterministic: Boolean)`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Thanks! @cloud-fan 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r128668731
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala ---
    @@ -130,460 +138,507 @@ 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 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 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       //////////////////////////////////////////////////////////////////////////////////////////////
       //////////////////////////////////////////////////////////////////////////////////////////////
     
       /**
    -   * 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
        * @param returnDataType  return type of udf. If it is null, spark would try to infer
        *                        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(
    --- End diff --
    
    then let's remove the `private[sql]` and add since tag


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF and Ja...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    **[Test build #79725 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79725/testReport)** for PR 17848 at commit [`9db97cb`](https://github.com/apache/spark/commit/9db97cbf0df28d6058f80b879d37641fdddb56f9).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Thanks! Merging to master.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    @zero323 
    - When `x` is non-deterministic, all the expressions that are derived from `x` (i.e., `y_i`, `z_i`, `v_i`) will be non-deterministic. 
    - When `x` is first materialized and computed, that means, the generated columns are deterministic. Thus, the results will be consistent.
    
    Not sure whether it answers your concern?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r128159780
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala ---
    @@ -58,6 +55,13 @@ case class UserDefinedFunction protected[sql] (
       def nullable: Boolean = _nullable
     
       /**
    +   * Returns true when the UDF is deterministic.
    --- End diff --
    
    s/when/iff/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r128159874
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala ---
    @@ -103,4 +110,19 @@ case class UserDefinedFunction protected[sql] (
           udf
         }
       }
    +
    +  /**
    +   * Updates UserDefinedFunction to non-deterministic.
    +   *
    +   * @since 2.3.0
    +   */
    +  def nonDeterministic(): UserDefinedFunction = {
    --- End diff --
    
    nondeterministic is a single word. Not nonDeterministic


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF and Ja...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79658/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF and Ja...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    **[Test build #79776 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79776/testReport)** for PR 17848 at commit [`d0a9086`](https://github.com/apache/spark/commit/d0a90865ca7c6a9afd6fbb28b3e8d1c9c602013c).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by zero323 <gi...@git.apache.org>.
Github user zero323 commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Disabling optimizations aside, to what extent can we actually support nondeterministic functions? Right now a common user mistake is to run RNG inside an UDF.  `nonDeterministic`could suggest it is fine, but I don't think we can guarantee this without reliable cache, can we?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic and distinc...

Posted by maropu <gi...@git.apache.org>.
Github user maropu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r115251937
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala ---
    @@ -36,17 +36,25 @@ import org.apache.spark.sql.types.DataType
      *                    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 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.
    --- End diff --
    
    Hive UDFs ignores `distinctLike` and AFAIK there is no optimisation rules for `distinctLike` though, do we need this `param` now?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic and distinc...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r116373710
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala ---
    @@ -144,11 +181,24 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends
         val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: Nil).toOption
         def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name))
         functionRegistry.registerFunction(name, builder)
    -    UserDefinedFunction(func, dataType, inputTypes)
    +    UserDefinedFunction(func, dataType, inputTypes, Some(name))
       }
     
       /**
    -   * Register a Scala closure of 2 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 2.3.0
    +   */
    +  def register[RT: TypeTag, A1: TypeTag](name: String, func: Function1[A1, RT], deterministic : Boolean, distinctLike: Boolean): UserDefinedFunction = {
    --- End diff --
    
    It is being used by Hive optimizer. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r128439969
  
    --- Diff: 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()`.
    --- End diff --
    
    let's also mention how to turn the UDF to be non-nullable.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r127866465
  
    --- Diff: sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java ---
    @@ -121,4 +122,29 @@ public void udf6Test() {
         Row result = spark.sql("SELECT returnOne()").head();
         Assert.assertEquals(1, result.getInt(0));
       }
    +
    +  public static class randUDFTest implements UDF1<Integer, Double> {
    --- End diff --
    
    `RandUDFTest`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF and Ja...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79725/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF and Ja...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    **[Test build #79776 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79776/testReport)** for PR 17848 at commit [`d0a9086`](https://github.com/apache/spark/commit/d0a90865ca7c6a9afd6fbb28b3e8d1c9c602013c).
     * This patch **fails to generate documentation**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    **[Test build #76922 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/76922/testReport)** for PR 17848 at commit [`d276b44`](https://github.com/apache/spark/commit/d276b44ce3f68344ae1151c930105fe291a925ec).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF and Ja...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79779/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic and distinc...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r116875422
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala ---
    @@ -115,6 +115,22 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends
               def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable)
               functionRegistry.registerFunction(name, builder)
               UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable)
    --- End diff --
    
    It will break the JAVA applications that call our Scala APIs with default arguments. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    **[Test build #76919 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/76919/testReport)** for PR 17848 at commit [`387af4b`](https://github.com/apache/spark/commit/387af4b98b3b32a89904d05678eb58d76852160c).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r128444612
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala ---
    @@ -130,460 +138,507 @@ 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 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 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       /**
    -   * Register 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 = {
         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, udfDeterministic = true)
         } 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)
    +    if (nullable) udf else udf.asNonNullabe()
       }
     
       //////////////////////////////////////////////////////////////////////////////////////////////
       //////////////////////////////////////////////////////////////////////////////////////////////
     
       /**
    -   * 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
        * @param returnDataType  return type of udf. If it is null, spark would try to infer
        *                        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(
    --- End diff --
    
    do we need a new method? it's private.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic and distinc...

Posted by maropu <gi...@git.apache.org>.
Github user maropu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r115253834
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala ---
    @@ -144,11 +181,24 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends
         val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: Nil).toOption
         def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name))
         functionRegistry.registerFunction(name, builder)
    -    UserDefinedFunction(func, dataType, inputTypes)
    +    UserDefinedFunction(func, dataType, inputTypes, Some(name))
       }
     
       /**
    -   * Register a Scala closure of 2 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 2.3.0
    +   */
    +  def register[RT: TypeTag, A1: TypeTag](name: String, func: Function1[A1, RT], deterministic : Boolean, distinctLike: Boolean): UserDefinedFunction = {
    --- End diff --
    
    I basically think these parameters are useful for users though, do we always need to set `deterministic` and `distinctLike` when registering UDFs? ISTM this is a little annoying for users, so we'd better to use default parameters for these parameters?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF and Ja...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79727/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r128141710
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala ---
    @@ -104,21 +104,35 @@ 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 = {
    +          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 = {
    --- End diff --
    
    can we make this public?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF and Ja...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic and distinc...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r116373714
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala ---
    @@ -36,17 +36,25 @@ import org.apache.spark.sql.types.DataType
      *                    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 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.
    --- End diff --
    
    It is being used by Hive optimizer.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    **[Test build #79228 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79228/testReport)** for PR 17848 at commit [`0c65322`](https://github.com/apache/spark/commit/0c65322f27cf9e1f5d686a45135798d1eb21a964).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r128159939
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala ---
    @@ -103,4 +110,19 @@ case class UserDefinedFunction protected[sql] (
           udf
         }
       }
    +
    +  /**
    +   * Updates UserDefinedFunction to non-deterministic.
    +   *
    +   * @since 2.3.0
    +   */
    +  def nonDeterministic(): UserDefinedFunction = {
    --- End diff --
    
    also udf.nondeterministic() doesn't exactly convey the meaning that this is mutable. I'd change it to 
    
    setNondeterministic


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r128162324
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala ---
    @@ -103,4 +110,19 @@ case class UserDefinedFunction protected[sql] (
           udf
         }
       }
    +
    +  /**
    +   * Updates UserDefinedFunction to non-deterministic.
    +   *
    +   * @since 2.3.0
    +   */
    +  def nonDeterministic(): UserDefinedFunction = {
    --- End diff --
    
    yea that works.
    
    or "asNondeterministic"



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF and Ja...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    **[Test build #79936 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79936/testReport)** for PR 17848 at commit [`bf060d6`](https://github.com/apache/spark/commit/bf060d66f62bcb54b7177af24a2dd7a9198b9864).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org

[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r127753508
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala ---
    @@ -103,4 +110,19 @@ case class UserDefinedFunction protected[sql] (
           udf
         }
       }
    +
    +  /**
    +   * Updates UserDefinedFunction to non-deterministic.
    +   *
    +   * @since 2.3.0
    +   */
    +  def nonDeterministic(): UserDefinedFunction = {
    --- End diff --
    
    The default is `deterministic`. When changing it to non-deterministic, users can call `nonDeterministic()`. 
    
    It might be rare to call `.withDeterministic(deterministic = true)`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r128668434
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala ---
    @@ -130,460 +138,507 @@ 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 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 = {
    --- End diff --
    
    as we have to add new APIs, why not we add a bunch of `def udf(f: UDF1[_, _]): UserDefinedFunction` instead of a bunch of `def register(name: String, f: UDF$i[$extTypeArgs], returnType: DataType, deterministic: Boolean)`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r128443096
  
    --- Diff: 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()`.
    --- End diff --
    
    a good example will be
    ```
    val foo = udf(() => { "hello" })
    spark.udf.register("stringContant", foo.asNonNullable())
    ```
    Although the return type of the UDF is `String` and is nullable, but we know that this UDF will never return null.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF and Ja...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    let's leave the java UDF API unchanged and think about whether we should add java UDF API in `functions` later.  @gatorsmile can you update the PR title? Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org

[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF and Ja...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by zero323 <gi...@git.apache.org>.
Github user zero323 commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    My concern is that people trying non-deterministic UDFs get tripped by repeated computations at least as often as by internal optimizations, and `nonDeterministic` flag might send a wrong message.
    
    In particular let's say we have this fan-out - fan-in worfklow depending on a non-deterministic source:
    
    ![image](https://cloud.githubusercontent.com/assets/1554276/26033144/64395fa0-38a5-11e7-9d0f-b2d6dbe51850.png)
    
    
    where dotted edges represent an arbitrary chain of transformations. Can we ensure that the state of each `foo`descendant in `sinl` will be consistent (`x` hasn't been recomputed)? I hope my point here is clear.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic and distinc...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r116395129
  
    --- Diff: sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java ---
    @@ -104,5 +105,36 @@ public void udf4Test() {
           sum += result.getLong(0);
         }
         Assert.assertEquals(55, sum);
    +    Assert.assertTrue("EXPLAIN outputs are expected to contain the UDF name.",
    +        spark.sql("EXPLAIN SELECT inc(1) AS f").collectAsList().toString().contains("inc"));
    --- End diff --
    
    This is to fix the issue of name loss for JavaUDF. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79935/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org

[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF and Ja...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    **[Test build #79658 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79658/testReport)** for PR 17848 at commit [`a336ffc`](https://github.com/apache/spark/commit/a336ffc1bd70050d82eef79f6179e25231b23aa1).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF and Ja...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    **[Test build #79725 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79725/testReport)** for PR 17848 at commit [`9db97cb`](https://github.com/apache/spark/commit/9db97cbf0df28d6058f80b879d37641fdddb56f9).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF and Ja...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    **[Test build #79779 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79779/testReport)** for PR 17848 at commit [`43bb9a9`](https://github.com/apache/spark/commit/43bb9a9254d0d694b2be57ec6a3574d53e9c3141).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/76915/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/76986/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF and Ja...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    @zero323 Which caches? Could you give an example? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79942/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic and distinctLike t...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/76918/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic and distinc...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r127456043
  
    --- Diff: python/pyspark/sql/context.py ---
    @@ -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):
    --- End diff --
    
    do we need `distinctLike` here? I think we only need it in `registerJavaUDAF`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/79936/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF and Ja...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    **[Test build #79780 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79780/testReport)** for PR 17848 at commit [`0ea4691`](https://github.com/apache/spark/commit/0ea4691d3ea979b86cb7c44f8290ff7dc805a8a7).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic and distinc...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r125387952
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala ---
    @@ -85,8 +94,9 @@ case class UserDefinedFunction protected[sql] (
        * @since 2.3.0
        */
       def withName(name: String): this.type = {
    -    this._nameOption = Option(name)
    -    this
    +    val udf = copyAll()
    +    udf._nameOption = Option(name)
    --- End diff --
    
    @maropu We should make a copy when calling `withName`, instead of returning this object. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17848: [SPARK-20586] [SQL] Add deterministic and distinc...

Posted by maropu <gi...@git.apache.org>.
Github user maropu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17848#discussion_r116857070
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala ---
    @@ -115,6 +115,22 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends
               def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name), nullable)
               functionRegistry.registerFunction(name, builder)
               UserDefinedFunction(func, dataType, inputTypes).withName(name).withNullability(nullable)
    --- End diff --
    
    We can't directly call `register(name, func, deterministic = true, distinctLike = false)` here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17848: [SPARK-20586] [SQL] Add deterministic to ScalaUDF and Ja...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/17848
  
    LGTM BTW.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org