Repository: spark
Updated Branches:
  refs/heads/master 82faacd79 -> 571635488


[SPARK-20918][SQL] Use FunctionIdentifier as function identifiers in 
FunctionRegistry

### What changes were proposed in this pull request?
Currently, the unquoted string of a function identifier is being used as the 
function identifier in the function registry. This could cause the incorrect 
the behavior when users use `.` in the function names. This PR is to take the 
`FunctionIdentifier` as the identifier in the function registry.

- Add one new function `createOrReplaceTempFunction` to `FunctionRegistry`
```Scala
final def createOrReplaceTempFunction(name: String, builder: FunctionBuilder): 
Unit
```

### How was this patch tested?
Add extra test cases to verify the inclusive bug fixes.

Author: Xiao Li <gatorsm...@gmail.com>
Author: gatorsmile <gatorsm...@gmail.com>

Closes #18142 from gatorsmile/fuctionRegistry.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/57163548
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/57163548
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/57163548

Branch: refs/heads/master
Commit: 571635488d6e16eee82f09ae0247c2f6ad5b7541
Parents: 82faacd
Author: Xiao Li <gatorsm...@gmail.com>
Authored: Fri Jun 9 10:16:30 2017 -0700
Committer: Wenchen Fan <wenc...@databricks.com>
Committed: Fri Jun 9 10:16:30 2017 -0700

----------------------------------------------------------------------
 .../catalyst/analysis/FunctionRegistry.scala    |  97 +++++++++++-------
 .../sql/catalyst/catalog/SessionCatalog.scala   |  37 ++++---
 .../catalyst/catalog/SessionCatalogSuite.scala  |   2 +-
 .../org/apache/spark/sql/UDFRegistration.scala  | 100 +++++++++----------
 .../spark/sql/execution/command/functions.scala |   2 +-
 .../spark/sql/GeneratorFunctionSuite.scala      |   3 +-
 .../org/apache/spark/sql/SQLQuerySuite.scala    |   2 +-
 .../apache/spark/sql/SessionStateSuite.scala    |   9 +-
 .../python/BatchEvalPythonExecSuite.scala       |   5 +-
 .../spark/sql/internal/CatalogSuite.scala       |   4 +
 .../spark/sql/hive/HiveSessionCatalog.scala     |   4 +-
 .../spark/sql/hive/execution/HiveUDFSuite.scala |  13 ++-
 .../sql/hive/execution/SQLQuerySuite.scala      |   2 +-
 13 files changed, 162 insertions(+), 118 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/57163548/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
----------------------------------------------------------------------
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
index 116b26f..4245b70 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
@@ -17,51 +17,68 @@
 
 package org.apache.spark.sql.catalyst.analysis
 
-import java.lang.reflect.Modifier
+import java.util.Locale
+import javax.annotation.concurrent.GuardedBy
 
+import scala.collection.mutable
 import scala.language.existentials
 import scala.reflect.ClassTag
 import scala.util.{Failure, Success, Try}
 
 import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.FunctionIdentifier
 import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.expressions.aggregate._
 import org.apache.spark.sql.catalyst.expressions.xml._
-import org.apache.spark.sql.catalyst.util.StringKeyHashMap
 import org.apache.spark.sql.types._
 
 
 /**
  * A catalog for looking up user defined functions, used by an [[Analyzer]].
  *
- * Note: The implementation should be thread-safe to allow concurrent access.
+ * Note:
+ *   1) The implementation should be thread-safe to allow concurrent access.
+ *   2) the database name is always case-sensitive here, callers are 
responsible to
+ *      format the database name w.r.t. case-sensitive config.
  */
 trait FunctionRegistry {
 
-  final def registerFunction(name: String, builder: FunctionBuilder): Unit = {
-    registerFunction(name, new 
ExpressionInfo(builder.getClass.getCanonicalName, name), builder)
+  final def registerFunction(name: FunctionIdentifier, builder: 
FunctionBuilder): Unit = {
+    val info = new ExpressionInfo(
+      builder.getClass.getCanonicalName, name.database.orNull, name.funcName)
+    registerFunction(name, info, builder)
   }
 
-  def registerFunction(name: String, info: ExpressionInfo, builder: 
FunctionBuilder): Unit
+  def registerFunction(
+    name: FunctionIdentifier,
+    info: ExpressionInfo,
+    builder: FunctionBuilder): Unit
+
+  /* Create or replace a temporary function. */
+  final def createOrReplaceTempFunction(name: String, builder: 
FunctionBuilder): Unit = {
+    registerFunction(
+      FunctionIdentifier(name),
+      builder)
+  }
 
   @throws[AnalysisException]("If function does not exist")
-  def lookupFunction(name: String, children: Seq[Expression]): Expression
+  def lookupFunction(name: FunctionIdentifier, children: Seq[Expression]): 
Expression
 
   /* List all of the registered function names. */
-  def listFunction(): Seq[String]
+  def listFunction(): Seq[FunctionIdentifier]
 
   /* Get the class of the registered function by specified name. */
-  def lookupFunction(name: String): Option[ExpressionInfo]
+  def lookupFunction(name: FunctionIdentifier): Option[ExpressionInfo]
 
   /* Get the builder of the registered function by specified name. */
-  def lookupFunctionBuilder(name: String): Option[FunctionBuilder]
+  def lookupFunctionBuilder(name: FunctionIdentifier): Option[FunctionBuilder]
 
   /** Drop a function and return whether the function existed. */
-  def dropFunction(name: String): Boolean
+  def dropFunction(name: FunctionIdentifier): Boolean
 
   /** Checks if a function with a given name exists. */
-  def functionExists(name: String): Boolean = lookupFunction(name).isDefined
+  def functionExists(name: FunctionIdentifier): Boolean = 
lookupFunction(name).isDefined
 
   /** Clear all registered functions. */
   def clear(): Unit
@@ -72,39 +89,47 @@ trait FunctionRegistry {
 
 class SimpleFunctionRegistry extends FunctionRegistry {
 
-  protected val functionBuilders =
-    StringKeyHashMap[(ExpressionInfo, FunctionBuilder)](caseSensitive = false)
+  @GuardedBy("this")
+  private val functionBuilders =
+    new mutable.HashMap[FunctionIdentifier, (ExpressionInfo, FunctionBuilder)]
+
+  // Resolution of the function name is always case insensitive, but the 
database name
+  // depends on the caller
+  private def normalizeFuncName(name: FunctionIdentifier): FunctionIdentifier 
= {
+    FunctionIdentifier(name.funcName.toLowerCase(Locale.ROOT), name.database)
+  }
 
   override def registerFunction(
-      name: String,
+      name: FunctionIdentifier,
       info: ExpressionInfo,
       builder: FunctionBuilder): Unit = synchronized {
-    functionBuilders.put(name, (info, builder))
+    functionBuilders.put(normalizeFuncName(name), (info, builder))
   }
 
-  override def lookupFunction(name: String, children: Seq[Expression]): 
Expression = {
+  override def lookupFunction(name: FunctionIdentifier, children: 
Seq[Expression]): Expression = {
     val func = synchronized {
-      functionBuilders.get(name).map(_._2).getOrElse {
+      functionBuilders.get(normalizeFuncName(name)).map(_._2).getOrElse {
         throw new AnalysisException(s"undefined function $name")
       }
     }
     func(children)
   }
 
-  override def listFunction(): Seq[String] = synchronized {
-    functionBuilders.iterator.map(_._1).toList.sorted
+  override def listFunction(): Seq[FunctionIdentifier] = synchronized {
+    functionBuilders.iterator.map(_._1).toList
   }
 
-  override def lookupFunction(name: String): Option[ExpressionInfo] = 
synchronized {
-    functionBuilders.get(name).map(_._1)
+  override def lookupFunction(name: FunctionIdentifier): 
Option[ExpressionInfo] = synchronized {
+    functionBuilders.get(normalizeFuncName(name)).map(_._1)
   }
 
-  override def lookupFunctionBuilder(name: String): Option[FunctionBuilder] = 
synchronized {
-    functionBuilders.get(name).map(_._2)
+  override def lookupFunctionBuilder(
+      name: FunctionIdentifier): Option[FunctionBuilder] = synchronized {
+    functionBuilders.get(normalizeFuncName(name)).map(_._2)
   }
 
-  override def dropFunction(name: String): Boolean = synchronized {
-    functionBuilders.remove(name).isDefined
+  override def dropFunction(name: FunctionIdentifier): Boolean = synchronized {
+    functionBuilders.remove(normalizeFuncName(name)).isDefined
   }
 
   override def clear(): Unit = synchronized {
@@ -125,28 +150,28 @@ class SimpleFunctionRegistry extends FunctionRegistry {
  * functions are already filled in and the analyzer needs only to resolve 
attribute references.
  */
 object EmptyFunctionRegistry extends FunctionRegistry {
-  override def registerFunction(name: String, info: ExpressionInfo, builder: 
FunctionBuilder)
-  : Unit = {
+  override def registerFunction(
+      name: FunctionIdentifier, info: ExpressionInfo, builder: 
FunctionBuilder): Unit = {
     throw new UnsupportedOperationException
   }
 
-  override def lookupFunction(name: String, children: Seq[Expression]): 
Expression = {
+  override def lookupFunction(name: FunctionIdentifier, children: 
Seq[Expression]): Expression = {
     throw new UnsupportedOperationException
   }
 
-  override def listFunction(): Seq[String] = {
+  override def listFunction(): Seq[FunctionIdentifier] = {
     throw new UnsupportedOperationException
   }
 
-  override def lookupFunction(name: String): Option[ExpressionInfo] = {
+  override def lookupFunction(name: FunctionIdentifier): 
Option[ExpressionInfo] = {
     throw new UnsupportedOperationException
   }
 
-  override def lookupFunctionBuilder(name: String): Option[FunctionBuilder] = {
+  override def lookupFunctionBuilder(name: FunctionIdentifier): 
Option[FunctionBuilder] = {
     throw new UnsupportedOperationException
   }
 
-  override def dropFunction(name: String): Boolean = {
+  override def dropFunction(name: FunctionIdentifier): Boolean = {
     throw new UnsupportedOperationException
   }
 
@@ -455,11 +480,13 @@ object FunctionRegistry {
 
   val builtin: SimpleFunctionRegistry = {
     val fr = new SimpleFunctionRegistry
-    expressions.foreach { case (name, (info, builder)) => 
fr.registerFunction(name, info, builder) }
+    expressions.foreach {
+      case (name, (info, builder)) => 
fr.registerFunction(FunctionIdentifier(name), info, builder)
+    }
     fr
   }
 
-  val functionSet: Set[String] = builtin.listFunction().toSet
+  val functionSet: Set[FunctionIdentifier] = builtin.listFunction().toSet
 
   /** See usage above. */
   private def expression[T <: Expression](name: String)

http://git-wip-us.apache.org/repos/asf/spark/blob/57163548/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
----------------------------------------------------------------------
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
index 57006bf..b6744a7 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
@@ -1029,13 +1029,12 @@ class SessionCatalog(
     requireDbExists(db)
     val identifier = name.copy(database = Some(db))
     if (functionExists(identifier)) {
-      // TODO: registry should just take in FunctionIdentifier for type safety
-      if (functionRegistry.functionExists(identifier.unquotedString)) {
+      if (functionRegistry.functionExists(identifier)) {
         // If we have loaded this function into the FunctionRegistry,
         // also drop it from there.
         // For a permanent function, because we loaded it to the 
FunctionRegistry
         // when it's first used, we also need to drop it from the 
FunctionRegistry.
-        functionRegistry.dropFunction(identifier.unquotedString)
+        functionRegistry.dropFunction(identifier)
       }
       externalCatalog.dropFunction(db, name.funcName)
     } else if (!ignoreIfNotExists) {
@@ -1061,7 +1060,7 @@ class SessionCatalog(
   def functionExists(name: FunctionIdentifier): Boolean = {
     val db = formatDatabaseName(name.database.getOrElse(getCurrentDatabase))
     requireDbExists(db)
-    functionRegistry.functionExists(name.unquotedString) ||
+    functionRegistry.functionExists(name) ||
       externalCatalog.functionExists(db, name.funcName)
   }
 
@@ -1095,20 +1094,20 @@ class SessionCatalog(
       ignoreIfExists: Boolean,
       functionBuilder: Option[FunctionBuilder] = None): Unit = {
     val func = funcDefinition.identifier
-    if (functionRegistry.functionExists(func.unquotedString) && 
!ignoreIfExists) {
+    if (functionRegistry.functionExists(func) && !ignoreIfExists) {
       throw new AnalysisException(s"Function $func already exists")
     }
     val info = new ExpressionInfo(funcDefinition.className, 
func.database.orNull, func.funcName)
     val builder =
       functionBuilder.getOrElse(makeFunctionBuilder(func.unquotedString, 
funcDefinition.className))
-    functionRegistry.registerFunction(func.unquotedString, info, builder)
+    functionRegistry.registerFunction(func, info, builder)
   }
 
   /**
    * Drop a temporary function.
    */
   def dropTempFunction(name: String, ignoreIfNotExists: Boolean): Unit = {
-    if (!functionRegistry.dropFunction(name) && !ignoreIfNotExists) {
+    if (!functionRegistry.dropFunction(FunctionIdentifier(name)) && 
!ignoreIfNotExists) {
       throw new NoSuchTempFunctionException(name)
     }
   }
@@ -1123,8 +1122,8 @@ class SessionCatalog(
     // A temporary function is a function that has been registered in 
functionRegistry
     // without a database name, and is neither a built-in function nor a Hive 
function
     name.database.isEmpty &&
-      functionRegistry.functionExists(name.funcName) &&
-      !FunctionRegistry.builtin.functionExists(name.funcName) &&
+      functionRegistry.functionExists(name) &&
+      !FunctionRegistry.builtin.functionExists(name) &&
       !hiveFunctions.contains(name.funcName.toLowerCase(Locale.ROOT))
   }
 
@@ -1140,8 +1139,8 @@ class SessionCatalog(
     // TODO: just make function registry take in FunctionIdentifier instead of 
duplicating this
     val database = 
name.database.orElse(Some(currentDb)).map(formatDatabaseName)
     val qualifiedName = name.copy(database = database)
-    functionRegistry.lookupFunction(name.funcName)
-      .orElse(functionRegistry.lookupFunction(qualifiedName.unquotedString))
+    functionRegistry.lookupFunction(name)
+      .orElse(functionRegistry.lookupFunction(qualifiedName))
       .getOrElse {
         val db = qualifiedName.database.get
         requireDbExists(db)
@@ -1176,19 +1175,19 @@ class SessionCatalog(
     // Note: the implementation of this function is a little bit convoluted.
     // We probably shouldn't use a single FunctionRegistry to register all 
three kinds of functions
     // (built-in, temp, and external).
-    if (name.database.isEmpty && 
functionRegistry.functionExists(name.funcName)) {
+    if (name.database.isEmpty && functionRegistry.functionExists(name)) {
       // This function has been already loaded into the function registry.
-      return functionRegistry.lookupFunction(name.funcName, children)
+      return functionRegistry.lookupFunction(name, children)
     }
 
     // If the name itself is not qualified, add the current database to it.
     val database = 
formatDatabaseName(name.database.getOrElse(getCurrentDatabase))
     val qualifiedName = name.copy(database = Some(database))
 
-    if (functionRegistry.functionExists(qualifiedName.unquotedString)) {
+    if (functionRegistry.functionExists(qualifiedName)) {
       // This function has been already loaded into the function registry.
       // Unlike the above block, we find this function by using the qualified 
name.
-      return functionRegistry.lookupFunction(qualifiedName.unquotedString, 
children)
+      return functionRegistry.lookupFunction(qualifiedName, children)
     }
 
     // The function has not been loaded to the function registry, which means
@@ -1209,7 +1208,7 @@ class SessionCatalog(
     // At here, we preserve the input from the user.
     registerFunction(catalogFunction.copy(identifier = qualifiedName), 
ignoreIfExists = false)
     // Now, we need to create the Expression.
-    functionRegistry.lookupFunction(qualifiedName.unquotedString, children)
+    functionRegistry.lookupFunction(qualifiedName, children)
   }
 
   /**
@@ -1229,8 +1228,8 @@ class SessionCatalog(
     requireDbExists(dbName)
     val dbFunctions = externalCatalog.listFunctions(dbName, pattern).map { f =>
       FunctionIdentifier(f, Some(dbName)) }
-    val loadedFunctions =
-      StringUtils.filterPattern(functionRegistry.listFunction(), pattern).map 
{ f =>
+    val loadedFunctions = StringUtils
+      .filterPattern(functionRegistry.listFunction().map(_.unquotedString), 
pattern).map { f =>
         // In functionRegistry, function names are stored as an unquoted 
format.
         Try(parser.parseFunctionIdentifier(f)) match {
           case Success(e) => e
@@ -1243,7 +1242,7 @@ class SessionCatalog(
     // The session catalog caches some persistent functions in the 
FunctionRegistry
     // so there can be duplicates.
     functions.map {
-      case f if FunctionRegistry.functionSet.contains(f.funcName) => (f, 
"SYSTEM")
+      case f if FunctionRegistry.functionSet.contains(f) => (f, "SYSTEM")
       case f => (f, "USER")
     }.distinct
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/57163548/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala
index be89030..5afeb0e 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala
@@ -1209,7 +1209,7 @@ abstract class SessionCatalogSuite extends PlanTest {
       assert(!catalog.isTemporaryFunction(FunctionIdentifier("func1")))
 
       // Returns false when the function is built-in or hive
-      assert(FunctionRegistry.builtin.functionExists("sum"))
+      
assert(FunctionRegistry.builtin.functionExists(FunctionIdentifier("sum")))
       assert(!catalog.isTemporaryFunction(FunctionIdentifier("sum")))
       
assert(!catalog.isTemporaryFunction(FunctionIdentifier("histogram_numeric")))
     }

http://git-wip-us.apache.org/repos/asf/spark/blob/57163548/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala
index 1bceac4..ad01b88 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala
@@ -61,7 +61,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
         | dataType: ${udf.dataType}
       """.stripMargin)
 
-    functionRegistry.registerFunction(name, udf.builder)
+    functionRegistry.createOrReplaceTempFunction(name, udf.builder)
   }
 
   /**
@@ -75,7 +75,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
    */
   def register(name: String, udaf: UserDefinedAggregateFunction): 
UserDefinedAggregateFunction = {
     def builder(children: Seq[Expression]) = ScalaUDAF(children, udaf)
-    functionRegistry.registerFunction(name, builder)
+    functionRegistry.createOrReplaceTempFunction(name, builder)
     udaf
   }
 
@@ -91,7 +91,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
    */
   def register(name: String, udf: UserDefinedFunction): UserDefinedFunction = {
     def builder(children: Seq[Expression]) = 
udf.apply(children.map(Column.apply) : _*).expr
-    functionRegistry.registerFunction(name, builder)
+    functionRegistry.createOrReplaceTempFunction(name, builder)
     udf
   }
 
@@ -113,7 +113,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
           val ScalaReflection.Schema(dataType, nullable) = 
ScalaReflection.schemaFor[RT]
           val inputTypes = Try($inputTypes).toOption
           def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, 
inputTypes.getOrElse(Nil), Some(name), nullable)
-          functionRegistry.registerFunction(name, builder)
+          functionRegistry.createOrReplaceTempFunction(name, builder)
           UserDefinedFunction(func, dataType, 
inputTypes).withName(name).withNullability(nullable)
         }""")
     }
@@ -130,7 +130,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
          | */
          |def register(name: String, f: UDF$i[$extTypeArgs, _], returnType: 
DataType): Unit = {
          |  val func = f$anyCast.call($anyParams)
-         |  functionRegistry.registerFunction(
+         |  functionRegistry.createOrReplaceTempFunction(
          |    name,
          |    (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
          |}""".stripMargin)
@@ -146,7 +146,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
     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)
-    functionRegistry.registerFunction(name, builder)
+    functionRegistry.createOrReplaceTempFunction(name, builder)
     UserDefinedFunction(func, dataType, 
inputTypes).withName(name).withNullability(nullable)
   }
 
@@ -159,7 +159,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
     val ScalaReflection.Schema(dataType, nullable) = 
ScalaReflection.schemaFor[RT]
     val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: 
Nil).toOption
     def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, 
inputTypes.getOrElse(Nil), Some(name), nullable)
-    functionRegistry.registerFunction(name, builder)
+    functionRegistry.createOrReplaceTempFunction(name, builder)
     UserDefinedFunction(func, dataType, 
inputTypes).withName(name).withNullability(nullable)
   }
 
@@ -172,7 +172,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
     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]) = ScalaUDF(func, dataType, e, 
inputTypes.getOrElse(Nil), Some(name), nullable)
-    functionRegistry.registerFunction(name, builder)
+    functionRegistry.createOrReplaceTempFunction(name, builder)
     UserDefinedFunction(func, dataType, 
inputTypes).withName(name).withNullability(nullable)
   }
 
@@ -185,7 +185,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
     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]) = ScalaUDF(func, dataType, e, 
inputTypes.getOrElse(Nil), Some(name), nullable)
-    functionRegistry.registerFunction(name, builder)
+    functionRegistry.createOrReplaceTempFunction(name, builder)
     UserDefinedFunction(func, dataType, 
inputTypes).withName(name).withNullability(nullable)
   }
 
@@ -198,7 +198,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
     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]) = ScalaUDF(func, dataType, e, 
inputTypes.getOrElse(Nil), Some(name), nullable)
-    functionRegistry.registerFunction(name, builder)
+    functionRegistry.createOrReplaceTempFunction(name, builder)
     UserDefinedFunction(func, dataType, 
inputTypes).withName(name).withNullability(nullable)
   }
 
@@ -211,7 +211,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
     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]) = ScalaUDF(func, dataType, e, 
inputTypes.getOrElse(Nil), Some(name), nullable)
-    functionRegistry.registerFunction(name, builder)
+    functionRegistry.createOrReplaceTempFunction(name, builder)
     UserDefinedFunction(func, dataType, 
inputTypes).withName(name).withNullability(nullable)
   }
 
@@ -224,7 +224,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
     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]) = ScalaUDF(func, dataType, e, 
inputTypes.getOrElse(Nil), Some(name), nullable)
-    functionRegistry.registerFunction(name, builder)
+    functionRegistry.createOrReplaceTempFunction(name, builder)
     UserDefinedFunction(func, dataType, 
inputTypes).withName(name).withNullability(nullable)
   }
 
@@ -237,7 +237,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
     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]) = ScalaUDF(func, dataType, e, 
inputTypes.getOrElse(Nil), Some(name), nullable)
-    functionRegistry.registerFunction(name, builder)
+    functionRegistry.createOrReplaceTempFunction(name, builder)
     UserDefinedFunction(func, dataType, 
inputTypes).withName(name).withNullability(nullable)
   }
 
@@ -250,7 +250,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
     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]) = ScalaUDF(func, dataType, e, 
inputTypes.getOrElse(Nil), Some(name), nullable)
-    functionRegistry.registerFunction(name, builder)
+    functionRegistry.createOrReplaceTempFunction(name, builder)
     UserDefinedFunction(func, dataType, 
inputTypes).withName(name).withNullability(nullable)
   }
 
@@ -263,7 +263,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
     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]) = ScalaUDF(func, dataType, e, 
inputTypes.getOrElse(Nil), Some(name), nullable)
-    functionRegistry.registerFunction(name, builder)
+    functionRegistry.createOrReplaceTempFunction(name, builder)
     UserDefinedFunction(func, dataType, 
inputTypes).withName(name).withNullability(nullable)
   }
 
@@ -276,7 +276,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
     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]) = ScalaUDF(func, dataType, e, 
inputTypes.getOrElse(Nil), Some(name), nullable)
-    functionRegistry.registerFunction(name, builder)
+    functionRegistry.createOrReplaceTempFunction(name, builder)
     UserDefinedFunction(func, dataType, 
inputTypes).withName(name).withNullability(nullable)
   }
 
@@ -289,7 +289,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
     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]) = ScalaUDF(func, dataType, e, 
inputTypes.getOrElse(Nil), Some(name), nullable)
-    functionRegistry.registerFunction(name, builder)
+    functionRegistry.createOrReplaceTempFunction(name, builder)
     UserDefinedFunction(func, dataType, 
inputTypes).withName(name).withNullability(nullable)
   }
 
@@ -302,7 +302,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
     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]) = ScalaUDF(func, dataType, e, 
inputTypes.getOrElse(Nil), Some(name), nullable)
-    functionRegistry.registerFunction(name, builder)
+    functionRegistry.createOrReplaceTempFunction(name, builder)
     UserDefinedFunction(func, dataType, 
inputTypes).withName(name).withNullability(nullable)
   }
 
@@ -315,7 +315,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
     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]) = ScalaUDF(func, dataType, e, 
inputTypes.getOrElse(Nil), Some(name), nullable)
-    functionRegistry.registerFunction(name, builder)
+    functionRegistry.createOrReplaceTempFunction(name, builder)
     UserDefinedFunction(func, dataType, 
inputTypes).withName(name).withNullability(nullable)
   }
 
@@ -328,7 +328,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
     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]) = ScalaUDF(func, dataType, e, 
inputTypes.getOrElse(Nil), Some(name), nullable)
-    functionRegistry.registerFunction(name, builder)
+    functionRegistry.createOrReplaceTempFunction(name, builder)
     UserDefinedFunction(func, dataType, 
inputTypes).withName(name).withNullability(nullable)
   }
 
@@ -341,7 +341,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
     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]) = ScalaUDF(func, dataType, e, 
inputTypes.getOrElse(Nil), Some(name), nullable)
-    functionRegistry.registerFunction(name, builder)
+    functionRegistry.createOrReplaceTempFunction(name, builder)
     UserDefinedFunction(func, dataType, 
inputTypes).withName(name).withNullability(nullable)
   }
 
@@ -354,7 +354,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
     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]) = ScalaUDF(func, dataType, e, 
inputTypes.getOrElse(Nil), Some(name), nullable)
-    functionRegistry.registerFunction(name, builder)
+    functionRegistry.createOrReplaceTempFunction(name, builder)
     UserDefinedFunction(func, dataType, 
inputTypes).withName(name).withNullability(nullable)
   }
 
@@ -367,7 +367,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
     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]) = ScalaUDF(func, dataType, e, 
inputTypes.getOrElse(Nil), Some(name), nullable)
-    functionRegistry.registerFunction(name, builder)
+    functionRegistry.createOrReplaceTempFunction(name, builder)
     UserDefinedFunction(func, dataType, 
inputTypes).withName(name).withNullability(nullable)
   }
 
@@ -380,7 +380,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
     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]) = ScalaUDF(func, dataType, e, 
inputTypes.getOrElse(Nil), Some(name), nullable)
-    functionRegistry.registerFunction(name, builder)
+    functionRegistry.createOrReplaceTempFunction(name, builder)
     UserDefinedFunction(func, dataType, 
inputTypes).withName(name).withNullability(nullable)
   }
 
@@ -393,7 +393,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
     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]) = ScalaUDF(func, dataType, e, 
inputTypes.getOrElse(Nil), Some(name), nullable)
-    functionRegistry.registerFunction(name, builder)
+    functionRegistry.createOrReplaceTempFunction(name, builder)
     UserDefinedFunction(func, dataType, 
inputTypes).withName(name).withNullability(nullable)
   }
 
@@ -406,7 +406,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
     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]) = ScalaUDF(func, dataType, e, 
inputTypes.getOrElse(Nil), Some(name), nullable)
-    functionRegistry.registerFunction(name, builder)
+    functionRegistry.createOrReplaceTempFunction(name, builder)
     UserDefinedFunction(func, dataType, 
inputTypes).withName(name).withNullability(nullable)
   }
 
@@ -419,7 +419,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
     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]) = ScalaUDF(func, dataType, e, 
inputTypes.getOrElse(Nil), Some(name), nullable)
-    functionRegistry.registerFunction(name, builder)
+    functionRegistry.createOrReplaceTempFunction(name, builder)
     UserDefinedFunction(func, dataType, 
inputTypes).withName(name).withNullability(nullable)
   }
 
@@ -432,7 +432,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
     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]) = ScalaUDF(func, dataType, e, 
inputTypes.getOrElse(Nil), Some(name), nullable)
-    functionRegistry.registerFunction(name, builder)
+    functionRegistry.createOrReplaceTempFunction(name, builder)
     UserDefinedFunction(func, dataType, 
inputTypes).withName(name).withNullability(nullable)
   }
 
@@ -510,7 +510,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
    */
   def register(name: String, f: UDF1[_, _], returnType: DataType): Unit = {
     val func = f.asInstanceOf[UDF1[Any, Any]].call(_: Any)
-    functionRegistry.registerFunction(
+    functionRegistry.createOrReplaceTempFunction(
       name,
       (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
   }
@@ -521,7 +521,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
    */
   def register(name: String, f: UDF2[_, _, _], returnType: DataType): Unit = {
     val func = f.asInstanceOf[UDF2[Any, Any, Any]].call(_: Any, _: Any)
-    functionRegistry.registerFunction(
+    functionRegistry.createOrReplaceTempFunction(
       name,
       (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
   }
@@ -532,7 +532,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
    */
   def register(name: String, f: UDF3[_, _, _, _], returnType: DataType): Unit 
= {
     val func = f.asInstanceOf[UDF3[Any, Any, Any, Any]].call(_: Any, _: Any, 
_: Any)
-    functionRegistry.registerFunction(
+    functionRegistry.createOrReplaceTempFunction(
       name,
       (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
   }
@@ -543,7 +543,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
    */
   def register(name: String, f: UDF4[_, _, _, _, _], returnType: DataType): 
Unit = {
     val func = f.asInstanceOf[UDF4[Any, Any, Any, Any, Any]].call(_: Any, _: 
Any, _: Any, _: Any)
-    functionRegistry.registerFunction(
+    functionRegistry.createOrReplaceTempFunction(
       name,
       (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
   }
@@ -554,7 +554,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
    */
   def register(name: String, f: UDF5[_, _, _, _, _, _], returnType: DataType): 
Unit = {
     val func = f.asInstanceOf[UDF5[Any, Any, Any, Any, Any, Any]].call(_: Any, 
_: Any, _: Any, _: Any, _: Any)
-    functionRegistry.registerFunction(
+    functionRegistry.createOrReplaceTempFunction(
       name,
       (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
   }
@@ -565,7 +565,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
    */
   def register(name: String, f: UDF6[_, _, _, _, _, _, _], returnType: 
DataType): Unit = {
     val func = f.asInstanceOf[UDF6[Any, Any, Any, Any, Any, Any, Any]].call(_: 
Any, _: Any, _: Any, _: Any, _: Any, _: Any)
-    functionRegistry.registerFunction(
+    functionRegistry.createOrReplaceTempFunction(
       name,
       (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
   }
@@ -576,7 +576,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
    */
   def register(name: String, f: UDF7[_, _, _, _, _, _, _, _], returnType: 
DataType): Unit = {
     val func = f.asInstanceOf[UDF7[Any, Any, Any, Any, Any, Any, Any, 
Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any)
-    functionRegistry.registerFunction(
+    functionRegistry.createOrReplaceTempFunction(
       name,
       (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
   }
@@ -587,7 +587,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
    */
   def register(name: String, f: UDF8[_, _, _, _, _, _, _, _, _], returnType: 
DataType): Unit = {
     val func = f.asInstanceOf[UDF8[Any, Any, Any, Any, Any, Any, Any, Any, 
Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any)
-    functionRegistry.registerFunction(
+    functionRegistry.createOrReplaceTempFunction(
       name,
       (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
   }
@@ -598,7 +598,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
    */
   def register(name: String, f: UDF9[_, _, _, _, _, _, _, _, _, _], 
returnType: DataType): Unit = {
     val func = f.asInstanceOf[UDF9[Any, Any, Any, Any, Any, Any, Any, Any, 
Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, 
_: Any)
-    functionRegistry.registerFunction(
+    functionRegistry.createOrReplaceTempFunction(
       name,
       (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
   }
@@ -609,7 +609,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
    */
   def register(name: String, f: UDF10[_, _, _, _, _, _, _, _, _, _, _], 
returnType: DataType): Unit = {
     val func = f.asInstanceOf[UDF10[Any, Any, Any, Any, Any, Any, Any, Any, 
Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: 
Any, _: Any, _: Any)
-    functionRegistry.registerFunction(
+    functionRegistry.createOrReplaceTempFunction(
       name,
       (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
   }
@@ -620,7 +620,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
    */
   def register(name: String, f: UDF11[_, _, _, _, _, _, _, _, _, _, _, _], 
returnType: DataType): Unit = {
     val func = f.asInstanceOf[UDF11[Any, Any, Any, Any, Any, Any, Any, Any, 
Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: 
Any, _: Any, _: Any, _: Any, _: Any)
-    functionRegistry.registerFunction(
+    functionRegistry.createOrReplaceTempFunction(
       name,
       (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
   }
@@ -631,7 +631,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
    */
   def register(name: String, f: UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _], 
returnType: DataType): Unit = {
     val func = f.asInstanceOf[UDF12[Any, Any, Any, Any, Any, Any, Any, Any, 
Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, 
_: Any, _: Any, _: Any, _: Any, _: Any, _: Any)
-    functionRegistry.registerFunction(
+    functionRegistry.createOrReplaceTempFunction(
       name,
       (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
   }
@@ -642,7 +642,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
    */
   def register(name: String, f: UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, 
_], returnType: DataType): Unit = {
     val func = f.asInstanceOf[UDF13[Any, Any, Any, Any, Any, Any, Any, Any, 
Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: 
Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any)
-    functionRegistry.registerFunction(
+    functionRegistry.createOrReplaceTempFunction(
       name,
       (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
   }
@@ -653,7 +653,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
    */
   def register(name: String, f: UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, 
_, _], returnType: DataType): Unit = {
     val func = f.asInstanceOf[UDF14[Any, Any, Any, Any, Any, Any, Any, Any, 
Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: 
Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any)
-    functionRegistry.registerFunction(
+    functionRegistry.createOrReplaceTempFunction(
       name,
       (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
   }
@@ -664,7 +664,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
    */
   def register(name: String, f: UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, 
_, _, _], returnType: DataType): Unit = {
     val func = f.asInstanceOf[UDF15[Any, Any, Any, Any, Any, Any, Any, Any, 
Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, 
_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, 
_: Any)
-    functionRegistry.registerFunction(
+    functionRegistry.createOrReplaceTempFunction(
       name,
       (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
   }
@@ -675,7 +675,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
    */
   def register(name: String, f: UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, 
_, _, _, _], returnType: DataType): Unit = {
     val func = f.asInstanceOf[UDF16[Any, Any, Any, Any, Any, Any, Any, Any, 
Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: 
Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: 
Any, _: Any, _: Any)
-    functionRegistry.registerFunction(
+    functionRegistry.createOrReplaceTempFunction(
       name,
       (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
   }
@@ -686,7 +686,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
    */
   def register(name: String, f: UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, 
_, _, _, _, _], returnType: DataType): Unit = {
     val func = f.asInstanceOf[UDF17[Any, Any, Any, Any, Any, Any, Any, Any, 
Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, 
_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, 
_: Any, _: Any, _: Any, _: Any)
-    functionRegistry.registerFunction(
+    functionRegistry.createOrReplaceTempFunction(
       name,
       (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
   }
@@ -697,7 +697,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
    */
   def register(name: String, f: UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, 
_, _, _, _, _, _], returnType: DataType): Unit = {
     val func = f.asInstanceOf[UDF18[Any, Any, Any, Any, Any, Any, Any, Any, 
Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: 
Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: 
Any, _: Any, _: Any, _: Any, _: Any, _: Any)
-    functionRegistry.registerFunction(
+    functionRegistry.createOrReplaceTempFunction(
       name,
       (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
   }
@@ -708,7 +708,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
    */
   def register(name: String, f: UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, 
_, _, _, _, _, _, _], returnType: DataType): Unit = {
     val func = f.asInstanceOf[UDF19[Any, Any, Any, Any, Any, Any, Any, Any, 
Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: 
Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: 
Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any)
-    functionRegistry.registerFunction(
+    functionRegistry.createOrReplaceTempFunction(
       name,
       (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
   }
@@ -719,7 +719,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
    */
   def register(name: String, f: UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, 
_, _, _, _, _, _, _, _], returnType: DataType): Unit = {
     val func = f.asInstanceOf[UDF20[Any, Any, Any, Any, Any, Any, Any, Any, 
Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, 
_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, 
_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any)
-    functionRegistry.registerFunction(
+    functionRegistry.createOrReplaceTempFunction(
       name,
       (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
   }
@@ -730,7 +730,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
    */
   def register(name: String, f: UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, 
_, _, _, _, _, _, _, _, _], returnType: DataType): Unit = {
     val func = f.asInstanceOf[UDF21[Any, Any, Any, Any, Any, Any, Any, Any, 
Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: 
Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: 
Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: 
Any)
-    functionRegistry.registerFunction(
+    functionRegistry.createOrReplaceTempFunction(
       name,
       (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
   }
@@ -741,7 +741,7 @@ class UDFRegistration private[sql] (functionRegistry: 
FunctionRegistry) extends
    */
   def register(name: String, f: UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, 
_, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = {
     val func = f.asInstanceOf[UDF22[Any, Any, Any, Any, Any, Any, Any, Any, 
Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, 
Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: 
Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: 
Any, _: Any, _: Any, _: Any)
-    functionRegistry.registerFunction(
+    functionRegistry.createOrReplaceTempFunction(
       name,
       (e: Seq[Expression]) => ScalaUDF(func, returnType, e))
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/57163548/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala
index 5450823..f39a326 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala
@@ -160,7 +160,7 @@ case class DropFunctionCommand(
         throw new AnalysisException(s"Specifying a database in DROP TEMPORARY 
FUNCTION " +
           s"is not allowed: '${databaseName.get}'")
       }
-      if (FunctionRegistry.builtin.functionExists(functionName)) {
+      if 
(FunctionRegistry.builtin.functionExists(FunctionIdentifier(functionName))) {
         throw new AnalysisException(s"Cannot drop native function 
'$functionName'")
       }
       catalog.dropTempFunction(functionName, ifExists)

http://git-wip-us.apache.org/repos/asf/spark/blob/57163548/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala
index b9871af..539c63d 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala
@@ -297,7 +297,8 @@ class GeneratorFunctionSuite extends QueryTest with 
SharedSQLContext {
   }
 
   test("outer generator()") {
-    spark.sessionState.functionRegistry.registerFunction("empty_gen", _ => 
EmptyGenerator())
+    spark.sessionState.functionRegistry
+      .createOrReplaceTempFunction("empty_gen", _ => EmptyGenerator())
     checkAnswer(
       sql("select * from values 1, 2 lateral view outer empty_gen() a as b"),
       Row(1, null) :: Row(2, null) :: Nil)

http://git-wip-us.apache.org/repos/asf/spark/blob/57163548/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
index 41e9e2c..a7efcaf 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
@@ -109,7 +109,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext 
{
 
   test("SPARK-14415: All functions should have own descriptions") {
     for (f <- spark.sessionState.functionRegistry.listFunction()) {
-      if (!Seq("cube", "grouping", "grouping_id", "rollup", 
"window").contains(f)) {
+      if (!Seq("cube", "grouping", "grouping_id", "rollup", 
"window").contains(f.unquotedString)) {
         checkKeywordsNotExist(sql(s"describe function `$f`"), "N/A.")
       }
     }

http://git-wip-us.apache.org/repos/asf/spark/blob/57163548/sql/core/src/test/scala/org/apache/spark/sql/SessionStateSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/SessionStateSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/SessionStateSuite.scala
index 5638c8e..c016667 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SessionStateSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SessionStateSuite.scala
@@ -22,6 +22,7 @@ import org.scalatest.BeforeAndAfterEach
 import scala.collection.mutable.ArrayBuffer
 
 import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.catalyst.FunctionIdentifier
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.catalyst.rules.Rule
 import org.apache.spark.sql.execution.QueryExecution
@@ -71,10 +72,10 @@ class SessionStateSuite extends SparkFunSuite
   }
 
   test("fork new session and inherit function registry and udf") {
-    val testFuncName1 = "strlenScala"
-    val testFuncName2 = "addone"
+    val testFuncName1 = FunctionIdentifier("strlenScala")
+    val testFuncName2 = FunctionIdentifier("addone")
     try {
-      activeSession.udf.register(testFuncName1, (_: String).length + (_: Int))
+      activeSession.udf.register(testFuncName1.funcName, (_: String).length + 
(_: Int))
       val forkedSession = activeSession.cloneSession()
 
       // inheritance
@@ -86,7 +87,7 @@ class SessionStateSuite extends SparkFunSuite
       // independence
       forkedSession.sessionState.functionRegistry.dropFunction(testFuncName1)
       
assert(activeSession.sessionState.functionRegistry.lookupFunction(testFuncName1).nonEmpty)
-      activeSession.udf.register(testFuncName2, (_: Int) + 1)
+      activeSession.udf.register(testFuncName2.funcName, (_: Int) + 1)
       
assert(forkedSession.sessionState.functionRegistry.lookupFunction(testFuncName2).isEmpty)
     } finally {
       activeSession.sessionState.functionRegistry.dropFunction(testFuncName1)

http://git-wip-us.apache.org/repos/asf/spark/blob/57163548/sql/core/src/test/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala
index 2a3d1cf..80ef4eb 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala
@@ -21,7 +21,8 @@ import scala.collection.JavaConverters._
 import scala.collection.mutable.ArrayBuffer
 
 import org.apache.spark.api.python.PythonFunction
-import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, 
EqualTo, Expression, GreaterThan, In}
+import org.apache.spark.sql.catalyst.FunctionIdentifier
+import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, 
GreaterThan, In}
 import org.apache.spark.sql.execution.{FilterExec, InputAdapter, 
SparkPlanTest, WholeStageCodegenExec}
 import org.apache.spark.sql.test.SharedSQLContext
 import org.apache.spark.sql.types.BooleanType
@@ -36,7 +37,7 @@ class BatchEvalPythonExecSuite extends SparkPlanTest with 
SharedSQLContext {
   }
 
   override def afterAll(): Unit = {
-    spark.sessionState.functionRegistry.dropFunction("dummyPythonUDF")
+    
spark.sessionState.functionRegistry.dropFunction(FunctionIdentifier("dummyPythonUDF"))
     super.afterAll()
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/57163548/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala
index bc641fd..b2d568c 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala
@@ -367,6 +367,7 @@ class CatalogSuite
       withUserDefinedFunction("fn1" -> true, s"$db.fn2" -> false) {
         // Try to find non existing functions.
         intercept[AnalysisException](spark.catalog.getFunction("fn1"))
+        intercept[AnalysisException](spark.catalog.getFunction(db, "fn1"))
         intercept[AnalysisException](spark.catalog.getFunction("fn2"))
         intercept[AnalysisException](spark.catalog.getFunction(db, "fn2"))
 
@@ -379,6 +380,8 @@ class CatalogSuite
         assert(fn1.name === "fn1")
         assert(fn1.database === null)
         assert(fn1.isTemporary)
+        // Find a temporary function with database
+        intercept[AnalysisException](spark.catalog.getFunction(db, "fn1"))
 
         // Find a qualified function
         val fn2 = spark.catalog.getFunction(db, "fn2")
@@ -455,6 +458,7 @@ class CatalogSuite
 
         // Find a temporary function
         assert(spark.catalog.functionExists("fn1"))
+        assert(!spark.catalog.functionExists(db, "fn1"))
 
         // Find a qualified function
         assert(spark.catalog.functionExists(db, "fn2"))

http://git-wip-us.apache.org/repos/asf/spark/blob/57163548/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala
index 6227e78..da87f02 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala
@@ -129,7 +129,7 @@ private[sql] class HiveSessionCatalog(
     Try(super.lookupFunction(funcName, children)) match {
       case Success(expr) => expr
       case Failure(error) =>
-        if (functionRegistry.functionExists(funcName.unquotedString)) {
+        if (functionRegistry.functionExists(funcName)) {
           // If the function actually exists in functionRegistry, it means 
that there is an
           // error when we create the Expression using the given children.
           // We need to throw the original exception.
@@ -163,7 +163,7 @@ private[sql] class HiveSessionCatalog(
           // Put this Hive built-in function to our function registry.
           registerFunction(func, ignoreIfExists = false)
           // Now, we need to create the Expression.
-          functionRegistry.lookupFunction(functionName, children)
+          functionRegistry.lookupFunction(functionIdentifier, children)
         }
     }
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/57163548/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala
index 8fcbad5..cae338c 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala
@@ -194,7 +194,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton 
with SQLTestUtils {
 
     checkAnswer(sql("SELECT percentile_approx(100.0D, array(0.9D, 0.9D)) FROM 
src LIMIT 1"),
       sql("SELECT array(100, 100) FROM src LIMIT 1").collect().toSeq)
-   }
+  }
 
   test("UDFIntegerToString") {
     val testData = spark.sparkContext.parallelize(
@@ -592,6 +592,17 @@ class HiveUDFSuite extends QueryTest with 
TestHiveSingleton with SQLTestUtils {
     }
   }
 
+  test("Temp function has dots in the names") {
+    withUserDefinedFunction("test_avg" -> false, "`default.test_avg`" -> true) 
{
+      sql(s"CREATE FUNCTION test_avg AS 
'${classOf[GenericUDAFAverage].getName}'")
+      checkAnswer(sql("SELECT test_avg(1)"), Row(1.0))
+      // temp function containing dots in the name
+      spark.udf.register("default.test_avg", () => { Math.random() + 2})
+      assert(sql("SELECT `default.test_avg`()").head().getDouble(0) >= 2.0)
+      checkAnswer(sql("SELECT test_avg(1)"), Row(1.0))
+    }
+  }
+
   test("Call the function registered in the not-current database") {
     Seq("true", "false").foreach { caseSensitive =>
       withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive) {

http://git-wip-us.apache.org/repos/asf/spark/blob/57163548/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
index da7a064..a949e5e 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
@@ -222,7 +222,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils 
with TestHiveSingleton {
   }
 
   test("show functions") {
-    val allBuiltinFunctions = 
FunctionRegistry.builtin.listFunction().toSet[String].toList.sorted
+    val allBuiltinFunctions = 
FunctionRegistry.builtin.listFunction().map(_.unquotedString)
     val allFunctions = sql("SHOW functions").collect().map(r => r(0))
     allBuiltinFunctions.foreach { f =>
       assert(allFunctions.contains(f))


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to