Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -1939,7 +1939,7 @@ class Analyzer(
def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
// Resolve functions with concrete relations from v2 catalog.
case UnresolvedFunc(multipartIdent) =>
val funcIdent = parseSessionCatalogFunctionIdentifier(multipartIdent, "function lookup")
val funcIdent = parseSessionCatalogFunctionIdentifier(multipartIdent)
ResolvedFunc(Identifier.of(funcIdent.database.toArray, funcIdent.funcName))

case q: LogicalPlan =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3600,7 +3600,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
} else {
Seq(describeFuncName.getText)
}
DescribeFunctionStatement(functionName, EXTENDED != null)
DescribeFunction(UnresolvedFunc(functionName), EXTENDED != null)
}

/**
Expand All @@ -3615,8 +3615,10 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
case Some(x) => throw new ParseException(s"SHOW $x FUNCTIONS not supported", ctx)
}
val pattern = Option(ctx.pattern).map(string(_))
val functionName = Option(ctx.multipartIdentifier).map(visitMultipartIdentifier)
ShowFunctionsStatement(userScope, systemScope, pattern, functionName)
val unresolvedFuncOpt = Option(ctx.multipartIdentifier)
.map(visitMultipartIdentifier)
.map(UnresolvedFunc(_))
ShowFunctions(unresolvedFuncOpt, userScope, systemScope, pattern)
}

/**
Expand All @@ -3629,8 +3631,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
*/
override def visitDropFunction(ctx: DropFunctionContext): LogicalPlan = withOrigin(ctx) {
val functionName = visitMultipartIdentifier(ctx.multipartIdentifier)
DropFunctionStatement(
functionName,
DropFunction(
UnresolvedFunc(functionName),
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

will analyzer give a nice error if UnresolvedFunc is left to CheckAnalysis?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nvm, this won't happen for now.

ctx.EXISTS != null,
ctx.TEMPORARY != null)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -446,30 +446,6 @@ case class ShowColumnsStatement(
*/
case class ShowCurrentNamespaceStatement() extends ParsedStatement

/**
* A DESCRIBE FUNCTION statement, as parsed from SQL
*/
case class DescribeFunctionStatement(
functionName: Seq[String],
isExtended: Boolean) extends ParsedStatement

/**
* SHOW FUNCTIONS statement, as parsed from SQL
*/
case class ShowFunctionsStatement(
userScope: Boolean,
systemScope: Boolean,
pattern: Option[String],
functionName: Option[Seq[String]]) extends ParsedStatement

/**
* DROP FUNCTION statement, as parsed from SQL
*/
case class DropFunctionStatement(
functionName: Seq[String],
ifExists: Boolean,
isTemp: Boolean) extends ParsedStatement

/**
* CREATE FUNCTION statement, as parsed from SQL
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -523,3 +523,31 @@ case class CommentOnTable(child: LogicalPlan, comment: String) extends Command {
case class RefreshFunction(child: LogicalPlan) extends Command {
override def children: Seq[LogicalPlan] = child :: Nil
}

/**
* The logical plan of the DESCRIBE FUNCTION command that works for v2 catalogs.
*/
case class DescribeFunction(child: LogicalPlan, isExtended: Boolean) extends Command {
override def children: Seq[LogicalPlan] = child :: Nil
}

/**
* The logical plan of the DROP FUNCTION command that works for v2 catalogs.
*/
case class DropFunction(
child: LogicalPlan,
ifExists: Boolean,
isTemp: Boolean) extends Command {
override def children: Seq[LogicalPlan] = child :: Nil
}

/**
* The logical plan of the SHOW FUNCTIONS command that works for v2 catalogs.
*/
case class ShowFunctions(
child: Option[LogicalPlan],
userScope: Boolean,
systemScope: Boolean,
pattern: Option[String]) extends Command {
override def children: Seq[LogicalPlan] = child.toSeq
}
Original file line number Diff line number Diff line change
Expand Up @@ -156,10 +156,7 @@ private[sql] trait LookupCatalog extends Logging {
}
}

// TODO: move function related v2 statements to the new framework.
def parseSessionCatalogFunctionIdentifier(
nameParts: Seq[String],
sql: String): FunctionIdentifier = {
def parseSessionCatalogFunctionIdentifier(nameParts: Seq[String]): FunctionIdentifier = {
if (nameParts.length == 1 && catalogManager.v1SessionCatalog.isTempFunction(nameParts.head)) {
return FunctionIdentifier(nameParts.head)
}
Expand All @@ -179,7 +176,7 @@ private[sql] trait LookupCatalog extends Logging {
}
}

case _ => throw new AnalysisException(s"$sql is only supported in v1 catalog")
case _ => throw new AnalysisException("function is only supported in v1 catalog")
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

function -> function command?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is called from Analyzer.scala except for CREATE FUNCTION, so I thought more general name would be better here. But if you think function command is better here, I am happy to change it. Please let me know!

}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -2026,60 +2026,60 @@ class DDLParserSuite extends AnalysisTest {
test("DESCRIBE FUNCTION") {
comparePlans(
parsePlan("DESC FUNCTION a"),
DescribeFunctionStatement(Seq("a"), false))
DescribeFunction(UnresolvedFunc(Seq("a")), false))
comparePlans(
parsePlan("DESCRIBE FUNCTION a"),
DescribeFunctionStatement(Seq("a"), false))
DescribeFunction(UnresolvedFunc(Seq("a")), false))
comparePlans(
parsePlan("DESCRIBE FUNCTION a.b.c"),
DescribeFunctionStatement(Seq("a", "b", "c"), false))
DescribeFunction(UnresolvedFunc(Seq("a", "b", "c")), false))
comparePlans(
parsePlan("DESCRIBE FUNCTION EXTENDED a.b.c"),
DescribeFunctionStatement(Seq("a", "b", "c"), true))
DescribeFunction(UnresolvedFunc(Seq("a", "b", "c")), true))
}

test("SHOW FUNCTIONS") {
comparePlans(
parsePlan("SHOW FUNCTIONS"),
ShowFunctionsStatement(true, true, None, None))
ShowFunctions(None, true, true, None))
comparePlans(
parsePlan("SHOW USER FUNCTIONS"),
ShowFunctionsStatement(true, false, None, None))
ShowFunctions(None, true, false, None))
comparePlans(
parsePlan("SHOW user FUNCTIONS"),
ShowFunctionsStatement(true, false, None, None))
ShowFunctions(None, true, false, None))
comparePlans(
parsePlan("SHOW SYSTEM FUNCTIONS"),
ShowFunctionsStatement(false, true, None, None))
ShowFunctions(None, false, true, None))
comparePlans(
parsePlan("SHOW ALL FUNCTIONS"),
ShowFunctionsStatement(true, true, None, None))
ShowFunctions(None, true, true, None))
comparePlans(
parsePlan("SHOW FUNCTIONS LIKE 'funct*'"),
ShowFunctionsStatement(true, true, Some("funct*"), None))
ShowFunctions(None, true, true, Some("funct*")))
comparePlans(
parsePlan("SHOW FUNCTIONS LIKE a.b.c"),
ShowFunctionsStatement(true, true, None, Some(Seq("a", "b", "c"))))
ShowFunctions(Some(UnresolvedFunc(Seq("a", "b", "c"))), true, true, None))
val sql = "SHOW other FUNCTIONS"
intercept(sql, s"$sql not supported")
}

test("DROP FUNCTION") {
comparePlans(
parsePlan("DROP FUNCTION a"),
DropFunctionStatement(Seq("a"), false, false))
DropFunction(UnresolvedFunc(Seq("a")), false, false))
comparePlans(
parsePlan("DROP FUNCTION a.b.c"),
DropFunctionStatement(Seq("a", "b", "c"), false, false))
DropFunction(UnresolvedFunc(Seq("a", "b", "c")), false, false))
comparePlans(
parsePlan("DROP TEMPORARY FUNCTION a.b.c"),
DropFunctionStatement(Seq("a", "b", "c"), false, true))
DropFunction(UnresolvedFunc(Seq("a", "b", "c")), false, true))
comparePlans(
parsePlan("DROP FUNCTION IF EXISTS a.b.c"),
DropFunctionStatement(Seq("a", "b", "c"), true, false))
DropFunction(UnresolvedFunc(Seq("a", "b", "c")), true, false))
comparePlans(
parsePlan("DROP TEMPORARY FUNCTION IF EXISTS a.b.c"),
DropFunctionStatement(Seq("a", "b", "c"), true, true))
DropFunction(UnresolvedFunc(Seq("a", "b", "c")), true, true))
}

test("CREATE FUNCTION") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -577,25 +577,20 @@ class ResolveSessionCatalog(
case ShowTableProperties(r: ResolvedView, propertyKey) =>
ShowTablePropertiesCommand(r.identifier.asTableIdentifier, propertyKey)

case DescribeFunctionStatement(nameParts, extended) =>
val functionIdent =
parseSessionCatalogFunctionIdentifier(nameParts, "DESCRIBE FUNCTION")
DescribeFunctionCommand(functionIdent, extended)

case ShowFunctionsStatement(userScope, systemScope, pattern, fun) =>
val (database, function) = fun match {
case Some(nameParts) =>
val FunctionIdentifier(fn, db) =
parseSessionCatalogFunctionIdentifier(nameParts, "SHOW FUNCTIONS")
(db, Some(fn))
case None => (None, pattern)
}
ShowFunctionsCommand(database, function, userScope, systemScope)
case DescribeFunction(ResolvedFunc(identifier), extended) =>
DescribeFunctionCommand(identifier.asFunctionIdentifier, extended)

case ShowFunctions(None, userScope, systemScope, pattern) =>
ShowFunctionsCommand(None, pattern, userScope, systemScope)

case DropFunctionStatement(nameParts, ifExists, isTemp) =>
val FunctionIdentifier(function, database) =
parseSessionCatalogFunctionIdentifier(nameParts, "DROP FUNCTION")
DropFunctionCommand(database, function, ifExists, isTemp)
case ShowFunctions(Some(ResolvedFunc(identifier)), userScope, systemScope, _) =>
val funcIdentifier = identifier.asFunctionIdentifier
ShowFunctionsCommand(
funcIdentifier.database, Some(funcIdentifier.funcName), userScope, systemScope)

case DropFunction(ResolvedFunc(identifier), ifExists, isTemp) =>
val funcIdentifier = identifier.asFunctionIdentifier
DropFunctionCommand(funcIdentifier.database, funcIdentifier.funcName, ifExists, isTemp)

case CreateFunctionStatement(nameParts,
className, resources, isTemp, ignoreIfExists, replace) =>
Expand All @@ -618,7 +613,7 @@ class ResolveSessionCatalog(
replace)
} else {
val FunctionIdentifier(function, database) =
parseSessionCatalogFunctionIdentifier(nameParts, "CREATE FUNCTION")
parseSessionCatalogFunctionIdentifier(nameParts)
CreateFunctionCommand(database, function, className, resources, isTemp, ignoreIfExists,
replace)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2185,7 +2185,7 @@ class DataSourceV2SQLSuite
val e = intercept[AnalysisException] {
sql("DESCRIBE FUNCTION testcat.ns1.ns2.fun")
}
assert(e.message.contains("DESCRIBE FUNCTION is only supported in v1 catalog"))
assert(e.message.contains("function is only supported in v1 catalog"))

val e1 = intercept[AnalysisException] {
sql("DESCRIBE FUNCTION default.ns1.ns2.fun")
Expand All @@ -2200,14 +2200,14 @@ class DataSourceV2SQLSuite
val e = intercept[AnalysisException] {
sql(s"SHOW FUNCTIONS LIKE $function")
}
assert(e.message.contains("SHOW FUNCTIONS is only supported in v1 catalog"))
assert(e.message.contains("function is only supported in v1 catalog"))
}

test("DROP FUNCTION: only support session catalog") {
val e = intercept[AnalysisException] {
sql("DROP FUNCTION testcat.ns1.ns2.fun")
}
assert(e.message.contains("DROP FUNCTION is only supported in v1 catalog"))
assert(e.message.contains("function is only supported in v1 catalog"))

val e1 = intercept[AnalysisException] {
sql("DROP FUNCTION default.ns1.ns2.fun")
Expand All @@ -2220,7 +2220,7 @@ class DataSourceV2SQLSuite
val e = intercept[AnalysisException] {
sql("CREATE FUNCTION testcat.ns1.ns2.fun as 'f'")
}
assert(e.message.contains("CREATE FUNCTION is only supported in v1 catalog"))
assert(e.message.contains("function is only supported in v1 catalog"))

val e1 = intercept[AnalysisException] {
sql("CREATE FUNCTION default.ns1.ns2.fun as 'f'")
Expand All @@ -2233,7 +2233,7 @@ class DataSourceV2SQLSuite
val e = intercept[AnalysisException] {
sql("REFRESH FUNCTION testcat.ns1.ns2.fun")
}
assert(e.message.contains("function lookup is only supported in v1 catalog"))
assert(e.message.contains("function is only supported in v1 catalog"))

val e1 = intercept[AnalysisException] {
sql("REFRESH FUNCTION default.ns1.ns2.fun")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -371,8 +371,8 @@ abstract class HiveComparisonTest extends SparkFunSuite with BeforeAndAfterAll {

// We will ignore the ExplainCommand, ShowFunctions, DescribeFunction
if ((!hiveQuery.logical.isInstanceOf[ExplainCommand]) &&
(!hiveQuery.logical.isInstanceOf[ShowFunctionsStatement]) &&
(!hiveQuery.logical.isInstanceOf[DescribeFunctionStatement]) &&
(!hiveQuery.logical.isInstanceOf[ShowFunctions]) &&
(!hiveQuery.logical.isInstanceOf[DescribeFunction]) &&
(!hiveQuery.logical.isInstanceOf[DescribeCommandBase]) &&
(!hiveQuery.logical.isInstanceOf[DescribeRelation]) &&
(!hiveQuery.logical.isInstanceOf[DescribeColumnStatement]) &&
Expand Down