You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by we...@apache.org on 2017/07/05 12:46:49 UTC
spark git commit: [SPARK-20383][SQL] Supporting Create [temporary]
Function with the keyword 'OR REPLACE' and 'IF NOT EXISTS'
Repository: spark
Updated Branches:
refs/heads/master 873f3ad2b -> 5787ace46
[SPARK-20383][SQL] Supporting Create [temporary] Function with the keyword 'OR REPLACE' and 'IF NOT EXISTS'
## What changes were proposed in this pull request?
support to create [temporary] function with the keyword 'OR REPLACE' and 'IF NOT EXISTS'
## How was this patch tested?
manual test and added test cases
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: ouyangxiaochen <ou...@zte.com.cn>
Closes #17681 from ouyangxiaochen/spark-419.
Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/5787ace4
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/5787ace4
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/5787ace4
Branch: refs/heads/master
Commit: 5787ace463b2abde50d2ca24e8dd111e3a7c158e
Parents: 873f3ad
Author: ouyangxiaochen <ou...@zte.com.cn>
Authored: Wed Jul 5 20:46:42 2017 +0800
Committer: Wenchen Fan <we...@databricks.com>
Committed: Wed Jul 5 20:46:42 2017 +0800
----------------------------------------------------------------------
.../apache/spark/sql/catalyst/parser/SqlBase.g4 | 3 +-
.../sql/catalyst/catalog/ExternalCatalog.scala | 9 ++++
.../sql/catalyst/catalog/InMemoryCatalog.scala | 6 +++
.../sql/catalyst/catalog/SessionCatalog.scala | 23 +++++++++
.../spark/sql/catalyst/catalog/events.scala | 10 ++++
.../catalog/ExternalCatalogEventSuite.scala | 9 ++++
.../catalyst/catalog/ExternalCatalogSuite.scala | 9 ++++
.../spark/sql/execution/SparkSqlParser.scala | 8 +--
.../spark/sql/execution/command/functions.scala | 46 ++++++++++++-----
.../sql/execution/command/DDLCommandSuite.scala | 52 +++++++++++++++++++-
.../spark/sql/execution/command/DDLSuite.scala | 51 +++++++++++++++++++
.../spark/sql/hive/HiveExternalCatalog.scala | 9 ++++
12 files changed, 216 insertions(+), 19 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/spark/blob/5787ace4/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
index 29f5544..ef9f88a 100644
--- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
+++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
@@ -126,7 +126,8 @@ statement
tableIdentifier ('(' colTypeList ')')? tableProvider
(OPTIONS tablePropertyList)? #createTempViewUsing
| ALTER VIEW tableIdentifier AS? query #alterViewQuery
- | CREATE TEMPORARY? FUNCTION qualifiedName AS className=STRING
+ | CREATE (OR REPLACE)? TEMPORARY? FUNCTION (IF NOT EXISTS)?
+ qualifiedName AS className=STRING
(USING resource (',' resource)*)? #createFunction
| DROP TEMPORARY? FUNCTION (IF EXISTS)? qualifiedName #dropFunction
| EXPLAIN (LOGICAL | FORMATTED | EXTENDED | CODEGEN | COST)?
http://git-wip-us.apache.org/repos/asf/spark/blob/5787ace4/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala
index 0254b6b..6000d48 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala
@@ -332,6 +332,15 @@ abstract class ExternalCatalog
protected def doDropFunction(db: String, funcName: String): Unit
+ final def alterFunction(db: String, funcDefinition: CatalogFunction): Unit = {
+ val name = funcDefinition.identifier.funcName
+ postToAll(AlterFunctionPreEvent(db, name))
+ doAlterFunction(db, funcDefinition)
+ postToAll(AlterFunctionEvent(db, name))
+ }
+
+ protected def doAlterFunction(db: String, funcDefinition: CatalogFunction): Unit
+
final def renameFunction(db: String, oldName: String, newName: String): Unit = {
postToAll(RenameFunctionPreEvent(db, oldName, newName))
doRenameFunction(db, oldName, newName)
http://git-wip-us.apache.org/repos/asf/spark/blob/5787ace4/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala
index 747190f..d253c72 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala
@@ -590,6 +590,12 @@ class InMemoryCatalog(
catalog(db).functions.remove(funcName)
}
+ override protected def doAlterFunction(db: String, func: CatalogFunction): Unit = synchronized {
+ requireDbExists(db)
+ requireFunctionExists(db, func.identifier.funcName)
+ catalog(db).functions.put(func.identifier.funcName, func)
+ }
+
override protected def doRenameFunction(
db: String,
oldName: String,
http://git-wip-us.apache.org/repos/asf/spark/blob/5787ace4/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 a86604e..c40d5f6 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
@@ -1056,6 +1056,29 @@ class SessionCatalog(
}
/**
+ * overwirte a metastore function in the database specified in `funcDefinition`..
+ * If no database is specified, assume the function is in the current database.
+ */
+ def alterFunction(funcDefinition: CatalogFunction): Unit = {
+ val db = formatDatabaseName(funcDefinition.identifier.database.getOrElse(getCurrentDatabase))
+ requireDbExists(db)
+ val identifier = FunctionIdentifier(funcDefinition.identifier.funcName, Some(db))
+ val newFuncDefinition = funcDefinition.copy(identifier = identifier)
+ if (functionExists(identifier)) {
+ 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)
+ }
+ externalCatalog.alterFunction(db, newFuncDefinition)
+ } else {
+ throw new NoSuchFunctionException(db = db, func = identifier.toString)
+ }
+ }
+
+ /**
* Retrieve the metadata of a metastore function.
*
* If a database is specified in `name`, this will return the function in that database.
http://git-wip-us.apache.org/repos/asf/spark/blob/5787ace4/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/events.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/events.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/events.scala
index 459973a..742a51e 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/events.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/events.scala
@@ -140,6 +140,16 @@ case class DropFunctionPreEvent(database: String, name: String) extends Function
case class DropFunctionEvent(database: String, name: String) extends FunctionEvent
/**
+ * Event fired before a function is altered.
+ */
+case class AlterFunctionPreEvent(database: String, name: String) extends FunctionEvent
+
+/**
+ * Event fired after a function has been altered.
+ */
+case class AlterFunctionEvent(database: String, name: String) extends FunctionEvent
+
+/**
* Event fired before a function is renamed.
*/
case class RenameFunctionPreEvent(
http://git-wip-us.apache.org/repos/asf/spark/blob/5787ace4/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogEventSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogEventSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogEventSuite.scala
index 2539ea6..087c26f 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogEventSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogEventSuite.scala
@@ -176,6 +176,15 @@ class ExternalCatalogEventSuite extends SparkFunSuite {
}
checkEvents(RenameFunctionPreEvent("db5", "fn7", "fn4") :: Nil)
+ // ALTER
+ val alteredFunctionDefinition = CatalogFunction(
+ identifier = FunctionIdentifier("fn4", Some("db5")),
+ className = "org.apache.spark.AlterFunction",
+ resources = Seq.empty)
+ catalog.alterFunction("db5", alteredFunctionDefinition)
+ checkEvents(
+ AlterFunctionPreEvent("db5", "fn4") :: AlterFunctionEvent("db5", "fn4") :: Nil)
+
// DROP
intercept[AnalysisException] {
catalog.dropFunction("db5", "fn7")
http://git-wip-us.apache.org/repos/asf/spark/blob/5787ace4/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala
index c22d55f..66e895a 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala
@@ -752,6 +752,14 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac
}
}
+ test("alter function") {
+ val catalog = newBasicCatalog()
+ assert(catalog.getFunction("db2", "func1").className == funcClass)
+ val myNewFunc = catalog.getFunction("db2", "func1").copy(className = newFuncClass)
+ catalog.alterFunction("db2", myNewFunc)
+ assert(catalog.getFunction("db2", "func1").className == newFuncClass)
+ }
+
test("list functions") {
val catalog = newBasicCatalog()
catalog.createFunction("db2", newFunc("func2"))
@@ -916,6 +924,7 @@ abstract class CatalogTestUtils {
lazy val partWithEmptyValue =
CatalogTablePartition(Map("a" -> "3", "b" -> ""), storageFormat)
lazy val funcClass = "org.apache.spark.myFunc"
+ lazy val newFuncClass = "org.apache.spark.myNewFunc"
/**
* Creates a basic catalog, with the following structure:
http://git-wip-us.apache.org/repos/asf/spark/blob/5787ace4/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala
index 2b79eb5..2f8e416 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala
@@ -687,8 +687,8 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
*
* For example:
* {{{
- * CREATE [TEMPORARY] FUNCTION [db_name.]function_name AS class_name
- * [USING JAR|FILE|ARCHIVE 'file_uri' [, JAR|FILE|ARCHIVE 'file_uri']];
+ * CREATE [OR REPLACE] [TEMPORARY] FUNCTION [IF NOT EXISTS] [db_name.]function_name
+ * AS class_name [USING JAR|FILE|ARCHIVE 'file_uri' [, JAR|FILE|ARCHIVE 'file_uri']];
* }}}
*/
override def visitCreateFunction(ctx: CreateFunctionContext): LogicalPlan = withOrigin(ctx) {
@@ -709,7 +709,9 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
functionIdentifier.funcName,
string(ctx.className),
resources,
- ctx.TEMPORARY != null)
+ ctx.TEMPORARY != null,
+ ctx.EXISTS != null,
+ ctx.REPLACE != null)
}
/**
http://git-wip-us.apache.org/repos/asf/spark/blob/5787ace4/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 a91ad41..4f92ffe 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
@@ -31,13 +31,13 @@ import org.apache.spark.sql.types.{StringType, StructField, StructType}
* The DDL command that creates a function.
* To create a temporary function, the syntax of using this command in SQL is:
* {{{
- * CREATE TEMPORARY FUNCTION functionName
+ * CREATE [OR REPLACE] TEMPORARY FUNCTION functionName
* AS className [USING JAR\FILE 'uri' [, JAR|FILE 'uri']]
* }}}
*
* To create a permanent function, the syntax in SQL is:
* {{{
- * CREATE FUNCTION [databaseName.]functionName
+ * CREATE [OR REPLACE] FUNCTION [IF NOT EXISTS] [databaseName.]functionName
* AS className [USING JAR\FILE 'uri' [, JAR|FILE 'uri']]
* }}}
*/
@@ -46,26 +46,46 @@ case class CreateFunctionCommand(
functionName: String,
className: String,
resources: Seq[FunctionResource],
- isTemp: Boolean)
+ isTemp: Boolean,
+ ifNotExists: Boolean,
+ replace: Boolean)
extends RunnableCommand {
+ if (ifNotExists && replace) {
+ throw new AnalysisException("CREATE FUNCTION with both IF NOT EXISTS and REPLACE" +
+ " is not allowed.")
+ }
+
+ // Disallow to define a temporary function with `IF NOT EXISTS`
+ if (ifNotExists && isTemp) {
+ throw new AnalysisException(
+ "It is not allowed to define a TEMPORARY function with IF NOT EXISTS.")
+ }
+
+ // Temporary function names should not contain database prefix like "database.function"
+ if (databaseName.isDefined && isTemp) {
+ throw new AnalysisException(s"Specifying a database in CREATE TEMPORARY FUNCTION " +
+ s"is not allowed: '${databaseName.get}'")
+ }
+
override def run(sparkSession: SparkSession): Seq[Row] = {
val catalog = sparkSession.sessionState.catalog
val func = CatalogFunction(FunctionIdentifier(functionName, databaseName), className, resources)
if (isTemp) {
- if (databaseName.isDefined) {
- throw new AnalysisException(s"Specifying a database in CREATE TEMPORARY FUNCTION " +
- s"is not allowed: '${databaseName.get}'")
- }
// We first load resources and then put the builder in the function registry.
catalog.loadFunctionResources(resources)
- catalog.registerFunction(func, overrideIfExists = false)
+ catalog.registerFunction(func, overrideIfExists = replace)
} else {
- // For a permanent, we will store the metadata into underlying external catalog.
- // This function will be loaded into the FunctionRegistry when a query uses it.
- // We do not load it into FunctionRegistry right now.
- // TODO: should we also parse "IF NOT EXISTS"?
- catalog.createFunction(func, ignoreIfExists = false)
+ // Handles `CREATE OR REPLACE FUNCTION AS ... USING ...`
+ if (replace && catalog.functionExists(func.identifier)) {
+ // alter the function in the metastore
+ catalog.alterFunction(CatalogFunction(func.identifier, className, resources))
+ } else {
+ // For a permanent, we will store the metadata into underlying external catalog.
+ // This function will be loaded into the FunctionRegistry when a query uses it.
+ // We do not load it into FunctionRegistry right now.
+ catalog.createFunction(CatalogFunction(func.identifier, className, resources), ifNotExists)
+ }
}
Seq.empty[Row]
}
http://git-wip-us.apache.org/repos/asf/spark/blob/5787ace4/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala
index 8a6bc62..5643c58 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala
@@ -181,8 +181,29 @@ class DDLCommandSuite extends PlanTest {
|'com.matthewrathbone.example.SimpleUDFExample' USING ARCHIVE '/path/to/archive',
|FILE '/path/to/file'
""".stripMargin
+ val sql3 =
+ """
+ |CREATE OR REPLACE TEMPORARY FUNCTION helloworld3 as
+ |'com.matthewrathbone.example.SimpleUDFExample' USING JAR '/path/to/jar1',
+ |JAR '/path/to/jar2'
+ """.stripMargin
+ val sql4 =
+ """
+ |CREATE OR REPLACE FUNCTION hello.world1 as
+ |'com.matthewrathbone.example.SimpleUDFExample' USING ARCHIVE '/path/to/archive',
+ |FILE '/path/to/file'
+ """.stripMargin
+ val sql5 =
+ """
+ |CREATE FUNCTION IF NOT EXISTS hello.world2 as
+ |'com.matthewrathbone.example.SimpleUDFExample' USING ARCHIVE '/path/to/archive',
+ |FILE '/path/to/file'
+ """.stripMargin
val parsed1 = parser.parsePlan(sql1)
val parsed2 = parser.parsePlan(sql2)
+ val parsed3 = parser.parsePlan(sql3)
+ val parsed4 = parser.parsePlan(sql4)
+ val parsed5 = parser.parsePlan(sql5)
val expected1 = CreateFunctionCommand(
None,
"helloworld",
@@ -190,7 +211,7 @@ class DDLCommandSuite extends PlanTest {
Seq(
FunctionResource(FunctionResourceType.fromString("jar"), "/path/to/jar1"),
FunctionResource(FunctionResourceType.fromString("jar"), "/path/to/jar2")),
- isTemp = true)
+ isTemp = true, ifNotExists = false, replace = false)
val expected2 = CreateFunctionCommand(
Some("hello"),
"world",
@@ -198,9 +219,36 @@ class DDLCommandSuite extends PlanTest {
Seq(
FunctionResource(FunctionResourceType.fromString("archive"), "/path/to/archive"),
FunctionResource(FunctionResourceType.fromString("file"), "/path/to/file")),
- isTemp = false)
+ isTemp = false, ifNotExists = false, replace = false)
+ val expected3 = CreateFunctionCommand(
+ None,
+ "helloworld3",
+ "com.matthewrathbone.example.SimpleUDFExample",
+ Seq(
+ FunctionResource(FunctionResourceType.fromString("jar"), "/path/to/jar1"),
+ FunctionResource(FunctionResourceType.fromString("jar"), "/path/to/jar2")),
+ isTemp = true, ifNotExists = false, replace = true)
+ val expected4 = CreateFunctionCommand(
+ Some("hello"),
+ "world1",
+ "com.matthewrathbone.example.SimpleUDFExample",
+ Seq(
+ FunctionResource(FunctionResourceType.fromString("archive"), "/path/to/archive"),
+ FunctionResource(FunctionResourceType.fromString("file"), "/path/to/file")),
+ isTemp = false, ifNotExists = false, replace = true)
+ val expected5 = CreateFunctionCommand(
+ Some("hello"),
+ "world2",
+ "com.matthewrathbone.example.SimpleUDFExample",
+ Seq(
+ FunctionResource(FunctionResourceType.fromString("archive"), "/path/to/archive"),
+ FunctionResource(FunctionResourceType.fromString("file"), "/path/to/file")),
+ isTemp = false, ifNotExists = true, replace = false)
comparePlans(parsed1, expected1)
comparePlans(parsed2, expected2)
+ comparePlans(parsed3, expected3)
+ comparePlans(parsed4, expected4)
+ comparePlans(parsed5, expected5)
}
test("drop function") {
http://git-wip-us.apache.org/repos/asf/spark/blob/5787ace4/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
index e4dd077..5c40d8b 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
@@ -2270,6 +2270,57 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils {
}
}
+ test("create temporary function with if not exists") {
+ withUserDefinedFunction("func1" -> true) {
+ val sql1 =
+ """
+ |CREATE TEMPORARY FUNCTION IF NOT EXISTS func1 as
+ |'com.matthewrathbone.example.SimpleUDFExample' USING JAR '/path/to/jar1',
+ |JAR '/path/to/jar2'
+ """.stripMargin
+ val e = intercept[AnalysisException] {
+ sql(sql1)
+ }.getMessage
+ assert(e.contains("It is not allowed to define a TEMPORARY function with IF NOT EXISTS"))
+ }
+ }
+
+ test("create function with both if not exists and replace") {
+ withUserDefinedFunction("func1" -> false) {
+ val sql1 =
+ """
+ |CREATE OR REPLACE FUNCTION IF NOT EXISTS func1 as
+ |'com.matthewrathbone.example.SimpleUDFExample' USING JAR '/path/to/jar1',
+ |JAR '/path/to/jar2'
+ """.stripMargin
+ val e = intercept[AnalysisException] {
+ sql(sql1)
+ }.getMessage
+ assert(e.contains("CREATE FUNCTION with both IF NOT EXISTS and REPLACE is not allowed"))
+ }
+ }
+
+ test("create temporary function by specifying a database") {
+ val dbName = "mydb"
+ withDatabase(dbName) {
+ sql(s"CREATE DATABASE $dbName")
+ sql(s"USE $dbName")
+ withUserDefinedFunction("func1" -> true) {
+ val sql1 =
+ s"""
+ |CREATE TEMPORARY FUNCTION $dbName.func1 as
+ |'com.matthewrathbone.example.SimpleUDFExample' USING JAR '/path/to/jar1',
+ |JAR '/path/to/jar2'
+ """.stripMargin
+ val e = intercept[AnalysisException] {
+ sql(sql1)
+ }.getMessage
+ assert(e.contains(s"Specifying a database in CREATE TEMPORARY FUNCTION " +
+ s"is not allowed: '$dbName'"))
+ }
+ }
+ }
+
Seq(true, false).foreach { caseSensitive =>
test(s"alter table add columns with existing column name - caseSensitive $caseSensitive") {
withSQLConf(SQLConf.CASE_SENSITIVE.key -> s"$caseSensitive") {
http://git-wip-us.apache.org/repos/asf/spark/blob/5787ace4/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala
index 2a17849..306b380 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala
@@ -1132,6 +1132,15 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
client.dropFunction(db, name)
}
+ override protected def doAlterFunction(
+ db: String, funcDefinition: CatalogFunction): Unit = withClient {
+ requireDbExists(db)
+ val functionName = funcDefinition.identifier.funcName.toLowerCase(Locale.ROOT)
+ requireFunctionExists(db, functionName)
+ val functionIdentifier = funcDefinition.identifier.copy(funcName = functionName)
+ client.alterFunction(db, funcDefinition.copy(identifier = functionIdentifier))
+ }
+
override protected def doRenameFunction(
db: String,
oldName: String,
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org