[SPARK-8867][SQL] Support list / describe function usage
As Hive does, we need to list all of the registered UDF and its usage for user. We add the annotation to describe a UDF, so we can get the literal description info while registering the UDF. e.g. ```scala ExpressionDescription( usage = "_FUNC_(expr) - Returns the absolute value of the numeric value", extended = """> SELECT _FUNC_('-1') 1""") case class Abs(child: Expression) extends UnaryArithmetic { ... ``` Author: Cheng Hao <hao.cheng@intel.com> Closes #7259 from chenghao-intel/desc_function and squashes the following commits: cf29bba [Cheng Hao] fixing the code style issue 5193855 [Cheng Hao] Add more powerful parser for show functions c645a6b [Cheng Hao] fix bug in unit test 78d40f1 [Cheng Hao] update the padding issue for usage 48ee4b3 [Cheng Hao] update as feedback 70eb4e9 [Cheng Hao] add show/describe function support
This commit is contained in:
parent
c025c3d0a1
commit
1efe97dc9e
|
@ -0,0 +1,43 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.spark.sql.catalyst.expressions;
|
||||
|
||||
import org.apache.spark.annotation.DeveloperApi;
|
||||
|
||||
import java.lang.annotation.Retention;
|
||||
import java.lang.annotation.RetentionPolicy;
|
||||
|
||||
/**
|
||||
* ::DeveloperApi::
|
||||
|
||||
* A function description type which can be recognized by FunctionRegistry, and will be used to
|
||||
* show the usage of the function in human language.
|
||||
*
|
||||
* `usage()` will be used for the function usage in brief way.
|
||||
* `extended()` will be used for the function usage in verbose way, suppose
|
||||
* an example will be provided.
|
||||
*
|
||||
* And we can refer the function name by `_FUNC_`, in `usage` and `extended`, as it's
|
||||
* registered in `FunctionRegistry`.
|
||||
*/
|
||||
@DeveloperApi
|
||||
@Retention(RetentionPolicy.RUNTIME)
|
||||
public @interface ExpressionDescription {
|
||||
String usage() default "_FUNC_ is undocumented";
|
||||
String extended() default "No example for _FUNC_.";
|
||||
}
|
|
@ -0,0 +1,55 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.spark.sql.catalyst.expressions;
|
||||
|
||||
/**
|
||||
* Expression information, will be used to describe a expression.
|
||||
*/
|
||||
public class ExpressionInfo {
|
||||
private String className;
|
||||
private String usage;
|
||||
private String name;
|
||||
private String extended;
|
||||
|
||||
public String getClassName() {
|
||||
return className;
|
||||
}
|
||||
|
||||
public String getUsage() {
|
||||
return usage;
|
||||
}
|
||||
|
||||
public String getName() {
|
||||
return name;
|
||||
}
|
||||
|
||||
public String getExtended() {
|
||||
return extended;
|
||||
}
|
||||
|
||||
public ExpressionInfo(String className, String name, String usage, String extended) {
|
||||
this.className = className;
|
||||
this.name = name;
|
||||
this.usage = usage;
|
||||
this.extended = extended;
|
||||
}
|
||||
|
||||
public ExpressionInfo(String className, String name) {
|
||||
this(className, name, null, null);
|
||||
}
|
||||
}
|
|
@ -30,26 +30,44 @@ import org.apache.spark.sql.catalyst.util.StringKeyHashMap
|
|||
/** A catalog for looking up user defined functions, used by an [[Analyzer]]. */
|
||||
trait FunctionRegistry {
|
||||
|
||||
def registerFunction(name: String, builder: FunctionBuilder): Unit
|
||||
final def registerFunction(name: String, builder: FunctionBuilder): Unit = {
|
||||
registerFunction(name, new ExpressionInfo(builder.getClass.getCanonicalName, name), builder)
|
||||
}
|
||||
|
||||
def registerFunction(name: String, info: ExpressionInfo, builder: FunctionBuilder): Unit
|
||||
|
||||
@throws[AnalysisException]("If function does not exist")
|
||||
def lookupFunction(name: String, children: Seq[Expression]): Expression
|
||||
|
||||
/* List all of the registered function names. */
|
||||
def listFunction(): Seq[String]
|
||||
|
||||
/* Get the class of the registered function by specified name. */
|
||||
def lookupFunction(name: String): Option[ExpressionInfo]
|
||||
}
|
||||
|
||||
class SimpleFunctionRegistry extends FunctionRegistry {
|
||||
|
||||
private val functionBuilders = StringKeyHashMap[FunctionBuilder](caseSensitive = false)
|
||||
private val functionBuilders =
|
||||
StringKeyHashMap[(ExpressionInfo, FunctionBuilder)](caseSensitive = false)
|
||||
|
||||
override def registerFunction(name: String, builder: FunctionBuilder): Unit = {
|
||||
functionBuilders.put(name, builder)
|
||||
override def registerFunction(name: String, info: ExpressionInfo, builder: FunctionBuilder)
|
||||
: Unit = {
|
||||
functionBuilders.put(name, (info, builder))
|
||||
}
|
||||
|
||||
override def lookupFunction(name: String, children: Seq[Expression]): Expression = {
|
||||
val func = functionBuilders.get(name).getOrElse {
|
||||
val func = functionBuilders.get(name).map(_._2).getOrElse {
|
||||
throw new AnalysisException(s"undefined function $name")
|
||||
}
|
||||
func(children)
|
||||
}
|
||||
|
||||
override def listFunction(): Seq[String] = functionBuilders.iterator.map(_._1).toList.sorted
|
||||
|
||||
override def lookupFunction(name: String): Option[ExpressionInfo] = {
|
||||
functionBuilders.get(name).map(_._1)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -57,13 +75,22 @@ 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, builder: FunctionBuilder): Unit = {
|
||||
override def registerFunction(name: String, info: ExpressionInfo, builder: FunctionBuilder)
|
||||
: Unit = {
|
||||
throw new UnsupportedOperationException
|
||||
}
|
||||
|
||||
override def lookupFunction(name: String, children: Seq[Expression]): Expression = {
|
||||
throw new UnsupportedOperationException
|
||||
}
|
||||
|
||||
override def listFunction(): Seq[String] = {
|
||||
throw new UnsupportedOperationException
|
||||
}
|
||||
|
||||
override def lookupFunction(name: String): Option[ExpressionInfo] = {
|
||||
throw new UnsupportedOperationException
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
@ -71,7 +98,7 @@ object FunctionRegistry {
|
|||
|
||||
type FunctionBuilder = Seq[Expression] => Expression
|
||||
|
||||
val expressions: Map[String, FunctionBuilder] = Map(
|
||||
val expressions: Map[String, (ExpressionInfo, FunctionBuilder)] = Map(
|
||||
// misc non-aggregate functions
|
||||
expression[Abs]("abs"),
|
||||
expression[CreateArray]("array"),
|
||||
|
@ -205,13 +232,13 @@ object FunctionRegistry {
|
|||
|
||||
val builtin: FunctionRegistry = {
|
||||
val fr = new SimpleFunctionRegistry
|
||||
expressions.foreach { case (name, builder) => fr.registerFunction(name, builder) }
|
||||
expressions.foreach { case (name, (info, builder)) => fr.registerFunction(name, info, builder) }
|
||||
fr
|
||||
}
|
||||
|
||||
/** See usage above. */
|
||||
private def expression[T <: Expression](name: String)
|
||||
(implicit tag: ClassTag[T]): (String, FunctionBuilder) = {
|
||||
(implicit tag: ClassTag[T]): (String, (ExpressionInfo, FunctionBuilder)) = {
|
||||
|
||||
// See if we can find a constructor that accepts Seq[Expression]
|
||||
val varargCtor = Try(tag.runtimeClass.getDeclaredConstructor(classOf[Seq[_]])).toOption
|
||||
|
@ -237,6 +264,15 @@ object FunctionRegistry {
|
|||
}
|
||||
}
|
||||
}
|
||||
(name, builder)
|
||||
|
||||
val clazz = tag.runtimeClass
|
||||
val df = clazz.getAnnotation(classOf[ExpressionDescription])
|
||||
if (df != null) {
|
||||
(name,
|
||||
(new ExpressionInfo(clazz.getCanonicalName, name, df.usage(), df.extended()),
|
||||
builder))
|
||||
} else {
|
||||
(name, (new ExpressionInfo(clazz.getCanonicalName, name), builder))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -65,6 +65,9 @@ case class UnaryPositive(child: Expression) extends UnaryExpression with Expects
|
|||
/**
|
||||
* A function that get the absolute value of the numeric value.
|
||||
*/
|
||||
@ExpressionDescription(
|
||||
usage = "_FUNC_(expr) - Returns the absolute value of the numeric value",
|
||||
extended = "> SELECT _FUNC_('-1');\n1")
|
||||
case class Abs(child: Expression)
|
||||
extends UnaryExpression with ExpectsInputTypes with CodegenFallback {
|
||||
|
||||
|
|
|
@ -214,6 +214,9 @@ trait String2StringExpression extends ImplicitCastInputTypes {
|
|||
/**
|
||||
* A function that converts the characters of a string to uppercase.
|
||||
*/
|
||||
@ExpressionDescription(
|
||||
usage = "_FUNC_(str) - Returns str with all characters changed to uppercase",
|
||||
extended = "> SELECT _FUNC_('SparkSql');\n 'SPARKSQL'")
|
||||
case class Upper(child: Expression)
|
||||
extends UnaryExpression with String2StringExpression {
|
||||
|
||||
|
@ -227,6 +230,9 @@ case class Upper(child: Expression)
|
|||
/**
|
||||
* A function that converts the characters of a string to lowercase.
|
||||
*/
|
||||
@ExpressionDescription(
|
||||
usage = "_FUNC_(str) - Returns str with all characters changed to lowercase",
|
||||
extended = "> SELECT _FUNC_('SparkSql');\n'sparksql'")
|
||||
case class Lower(child: Expression) extends UnaryExpression with String2StringExpression {
|
||||
|
||||
override def convert(v: UTF8String): UTF8String = v.toLowerCase
|
||||
|
|
|
@ -17,7 +17,8 @@
|
|||
|
||||
package org.apache.spark.sql.catalyst.plans.logical
|
||||
|
||||
import org.apache.spark.sql.catalyst.expressions.Attribute
|
||||
import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Attribute}
|
||||
import org.apache.spark.sql.types.StringType
|
||||
|
||||
/**
|
||||
* A logical node that represents a non-query command to be executed by the system. For example,
|
||||
|
@ -25,3 +26,28 @@ import org.apache.spark.sql.catalyst.expressions.Attribute
|
|||
* eagerly executed.
|
||||
*/
|
||||
trait Command
|
||||
|
||||
/**
|
||||
* Returned for the "DESCRIBE [EXTENDED] FUNCTION functionName" command.
|
||||
* @param functionName The function to be described.
|
||||
* @param isExtended True if "DESCRIBE EXTENDED" is used. Otherwise, false.
|
||||
*/
|
||||
private[sql] case class DescribeFunction(
|
||||
functionName: String,
|
||||
isExtended: Boolean) extends LogicalPlan with Command {
|
||||
|
||||
override def children: Seq[LogicalPlan] = Seq.empty
|
||||
override val output: Seq[Attribute] = Seq(
|
||||
AttributeReference("function_desc", StringType, nullable = false)())
|
||||
}
|
||||
|
||||
/**
|
||||
* Returned for the "SHOW FUNCTIONS" command, which will list all of the
|
||||
* registered function list.
|
||||
*/
|
||||
private[sql] case class ShowFunctions(
|
||||
db: Option[String], pattern: Option[String]) extends LogicalPlan with Command {
|
||||
override def children: Seq[LogicalPlan] = Seq.empty
|
||||
override val output: Seq[Attribute] = Seq(
|
||||
AttributeReference("function", StringType, nullable = false)())
|
||||
}
|
||||
|
|
|
@ -21,7 +21,7 @@ import scala.util.parsing.combinator.RegexParsers
|
|||
|
||||
import org.apache.spark.sql.catalyst.AbstractSparkSQLParser
|
||||
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
|
||||
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
|
||||
import org.apache.spark.sql.catalyst.plans.logical.{DescribeFunction, LogicalPlan, ShowFunctions}
|
||||
import org.apache.spark.sql.execution._
|
||||
import org.apache.spark.sql.types.StringType
|
||||
|
||||
|
@ -57,6 +57,10 @@ private[sql] class SparkSQLParser(fallback: String => LogicalPlan) extends Abstr
|
|||
protected val AS = Keyword("AS")
|
||||
protected val CACHE = Keyword("CACHE")
|
||||
protected val CLEAR = Keyword("CLEAR")
|
||||
protected val DESCRIBE = Keyword("DESCRIBE")
|
||||
protected val EXTENDED = Keyword("EXTENDED")
|
||||
protected val FUNCTION = Keyword("FUNCTION")
|
||||
protected val FUNCTIONS = Keyword("FUNCTIONS")
|
||||
protected val IN = Keyword("IN")
|
||||
protected val LAZY = Keyword("LAZY")
|
||||
protected val SET = Keyword("SET")
|
||||
|
@ -65,7 +69,8 @@ private[sql] class SparkSQLParser(fallback: String => LogicalPlan) extends Abstr
|
|||
protected val TABLES = Keyword("TABLES")
|
||||
protected val UNCACHE = Keyword("UNCACHE")
|
||||
|
||||
override protected lazy val start: Parser[LogicalPlan] = cache | uncache | set | show | others
|
||||
override protected lazy val start: Parser[LogicalPlan] =
|
||||
cache | uncache | set | show | desc | others
|
||||
|
||||
private lazy val cache: Parser[LogicalPlan] =
|
||||
CACHE ~> LAZY.? ~ (TABLE ~> ident) ~ (AS ~> restInput).? ^^ {
|
||||
|
@ -85,9 +90,24 @@ private[sql] class SparkSQLParser(fallback: String => LogicalPlan) extends Abstr
|
|||
case input => SetCommandParser(input)
|
||||
}
|
||||
|
||||
// It can be the following patterns:
|
||||
// SHOW FUNCTIONS;
|
||||
// SHOW FUNCTIONS mydb.func1;
|
||||
// SHOW FUNCTIONS func1;
|
||||
// SHOW FUNCTIONS `mydb.a`.`func1.aa`;
|
||||
private lazy val show: Parser[LogicalPlan] =
|
||||
SHOW ~> TABLES ~ (IN ~> ident).? ^^ {
|
||||
case _ ~ dbName => ShowTablesCommand(dbName)
|
||||
( SHOW ~> TABLES ~ (IN ~> ident).? ^^ {
|
||||
case _ ~ dbName => ShowTablesCommand(dbName)
|
||||
}
|
||||
| SHOW ~ FUNCTIONS ~> ((ident <~ ".").? ~ (ident | stringLit)).? ^^ {
|
||||
case Some(f) => ShowFunctions(f._1, Some(f._2))
|
||||
case None => ShowFunctions(None, None)
|
||||
}
|
||||
)
|
||||
|
||||
private lazy val desc: Parser[LogicalPlan] =
|
||||
DESCRIBE ~ FUNCTION ~> EXTENDED.? ~ (ident | stringLit) ^^ {
|
||||
case isExtended ~ functionName => DescribeFunction(functionName, isExtended.isDefined)
|
||||
}
|
||||
|
||||
private lazy val others: Parser[LogicalPlan] =
|
||||
|
|
|
@ -428,6 +428,11 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
|
|||
ExecutedCommand(
|
||||
RunnableDescribeCommand(resultPlan, describe.output, isExtended)) :: Nil
|
||||
|
||||
case logical.ShowFunctions(db, pattern) => ExecutedCommand(ShowFunctions(db, pattern)) :: Nil
|
||||
|
||||
case logical.DescribeFunction(function, extended) =>
|
||||
ExecutedCommand(DescribeFunction(function, extended)) :: Nil
|
||||
|
||||
case _ => Nil
|
||||
}
|
||||
}
|
||||
|
|
|
@ -24,7 +24,7 @@ import org.apache.spark.annotation.DeveloperApi
|
|||
import org.apache.spark.rdd.RDD
|
||||
import org.apache.spark.sql.catalyst.{InternalRow, CatalystTypeConverters}
|
||||
import org.apache.spark.sql.catalyst.errors.TreeNodeException
|
||||
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
|
||||
import org.apache.spark.sql.catalyst.expressions.{ExpressionDescription, Expression, Attribute, AttributeReference}
|
||||
import org.apache.spark.sql.catalyst.plans.logical
|
||||
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
|
||||
import org.apache.spark.sql.types._
|
||||
|
@ -298,3 +298,78 @@ case class ShowTablesCommand(databaseName: Option[String]) extends RunnableComma
|
|||
rows
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A command for users to list all of the registered functions.
|
||||
* The syntax of using this command in SQL is:
|
||||
* {{{
|
||||
* SHOW FUNCTIONS
|
||||
* }}}
|
||||
* TODO currently we are simply ignore the db
|
||||
*/
|
||||
case class ShowFunctions(db: Option[String], pattern: Option[String]) extends RunnableCommand {
|
||||
override val output: Seq[Attribute] = {
|
||||
val schema = StructType(
|
||||
StructField("function", StringType, nullable = false) :: Nil)
|
||||
|
||||
schema.toAttributes
|
||||
}
|
||||
|
||||
override def run(sqlContext: SQLContext): Seq[Row] = pattern match {
|
||||
case Some(p) =>
|
||||
try {
|
||||
val regex = java.util.regex.Pattern.compile(p)
|
||||
sqlContext.functionRegistry.listFunction().filter(regex.matcher(_).matches()).map(Row(_))
|
||||
} catch {
|
||||
// probably will failed in the regex that user provided, then returns empty row.
|
||||
case _: Throwable => Seq.empty[Row]
|
||||
}
|
||||
case None =>
|
||||
sqlContext.functionRegistry.listFunction().map(Row(_))
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A command for users to get the usage of a registered function.
|
||||
* The syntax of using this command in SQL is
|
||||
* {{{
|
||||
* DESCRIBE FUNCTION [EXTENDED] upper;
|
||||
* }}}
|
||||
*/
|
||||
case class DescribeFunction(
|
||||
functionName: String,
|
||||
isExtended: Boolean) extends RunnableCommand {
|
||||
|
||||
override val output: Seq[Attribute] = {
|
||||
val schema = StructType(
|
||||
StructField("function_desc", StringType, nullable = false) :: Nil)
|
||||
|
||||
schema.toAttributes
|
||||
}
|
||||
|
||||
private def replaceFunctionName(usage: String, functionName: String): String = {
|
||||
if (usage == null) {
|
||||
"To be added."
|
||||
} else {
|
||||
usage.replaceAll("_FUNC_", functionName)
|
||||
}
|
||||
}
|
||||
|
||||
override def run(sqlContext: SQLContext): Seq[Row] = {
|
||||
sqlContext.functionRegistry.lookupFunction(functionName) match {
|
||||
case Some(info) =>
|
||||
val result =
|
||||
Row(s"Function: ${info.getName}") ::
|
||||
Row(s"Class: ${info.getClassName}") ::
|
||||
Row(s"Usage: ${replaceFunctionName(info.getUsage(), info.getName)}") :: Nil
|
||||
|
||||
if (isExtended) {
|
||||
result :+ Row(s"Extended Usage:\n${replaceFunctionName(info.getExtended, info.getName)}")
|
||||
} else {
|
||||
result
|
||||
}
|
||||
|
||||
case None => Seq(Row(s"Function: $functionName is not found."))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
|
||||
package org.apache.spark.sql
|
||||
|
||||
import org.apache.spark.sql.catalyst.analysis.FunctionRegistry
|
||||
import org.scalatest.BeforeAndAfterAll
|
||||
|
||||
import java.sql.Timestamp
|
||||
|
@ -58,6 +59,31 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll with SQLTestUtils {
|
|||
checkAnswer(queryCoalesce, Row("1") :: Nil)
|
||||
}
|
||||
|
||||
test("show functions") {
|
||||
checkAnswer(sql("SHOW functions"), FunctionRegistry.builtin.listFunction().sorted.map(Row(_)))
|
||||
}
|
||||
|
||||
test("describe functions") {
|
||||
checkExistence(sql("describe function extended upper"), true,
|
||||
"Function: upper",
|
||||
"Class: org.apache.spark.sql.catalyst.expressions.Upper",
|
||||
"Usage: upper(str) - Returns str with all characters changed to uppercase",
|
||||
"Extended Usage:",
|
||||
"> SELECT upper('SparkSql');",
|
||||
"'SPARKSQL'")
|
||||
|
||||
checkExistence(sql("describe functioN Upper"), true,
|
||||
"Function: upper",
|
||||
"Class: org.apache.spark.sql.catalyst.expressions.Upper",
|
||||
"Usage: upper(str) - Returns str with all characters changed to uppercase")
|
||||
|
||||
checkExistence(sql("describe functioN Upper"), false,
|
||||
"Extended Usage")
|
||||
|
||||
checkExistence(sql("describe functioN abcadf"), true,
|
||||
"Function: abcadf is not found.")
|
||||
}
|
||||
|
||||
test("SPARK-6743: no columns from cache") {
|
||||
Seq(
|
||||
(83, 0, 38),
|
||||
|
|
|
@ -76,8 +76,32 @@ private[hive] class HiveFunctionRegistry(underlying: analysis.FunctionRegistry)
|
|||
}
|
||||
}
|
||||
|
||||
override def registerFunction(name: String, builder: FunctionBuilder): Unit =
|
||||
underlying.registerFunction(name, builder)
|
||||
override def registerFunction(name: String, info: ExpressionInfo, builder: FunctionBuilder)
|
||||
: Unit = underlying.registerFunction(name, info, builder)
|
||||
|
||||
/* List all of the registered function names. */
|
||||
override def listFunction(): Seq[String] = {
|
||||
val a = FunctionRegistry.getFunctionNames ++ underlying.listFunction()
|
||||
a.toList.sorted
|
||||
}
|
||||
|
||||
/* Get the class of the registered function by specified name. */
|
||||
override def lookupFunction(name: String): Option[ExpressionInfo] = {
|
||||
underlying.lookupFunction(name).orElse(
|
||||
Try {
|
||||
val info = FunctionRegistry.getFunctionInfo(name)
|
||||
val annotation = info.getFunctionClass.getAnnotation(classOf[Description])
|
||||
if (annotation != null) {
|
||||
Some(new ExpressionInfo(
|
||||
info.getFunctionClass.getCanonicalName,
|
||||
annotation.name(),
|
||||
annotation.value(),
|
||||
annotation.extended()))
|
||||
} else {
|
||||
None
|
||||
}
|
||||
}.getOrElse(None))
|
||||
}
|
||||
}
|
||||
|
||||
private[hive] case class HiveSimpleUDF(funcWrapper: HiveFunctionWrapper, children: Seq[Expression])
|
||||
|
|
|
@ -370,7 +370,11 @@ abstract class HiveComparisonTest
|
|||
// Check that the results match unless its an EXPLAIN query.
|
||||
val preparedHive = prepareAnswer(hiveQuery, hive)
|
||||
|
||||
if ((!hiveQuery.logical.isInstanceOf[ExplainCommand]) && preparedHive != catalyst) {
|
||||
// We will ignore the ExplainCommand, ShowFunctions, DescribeFunction
|
||||
if ((!hiveQuery.logical.isInstanceOf[ExplainCommand]) &&
|
||||
(!hiveQuery.logical.isInstanceOf[ShowFunctions]) &&
|
||||
(!hiveQuery.logical.isInstanceOf[DescribeFunction]) &&
|
||||
preparedHive != catalyst) {
|
||||
|
||||
val hivePrintOut = s"== HIVE - ${preparedHive.size} row(s) ==" +: preparedHive
|
||||
val catalystPrintOut = s"== CATALYST - ${catalyst.size} row(s) ==" +: catalyst
|
||||
|
|
|
@ -19,9 +19,11 @@ package org.apache.spark.sql.hive.execution
|
|||
|
||||
import java.sql.{Date, Timestamp}
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
|
||||
import org.apache.spark.sql._
|
||||
import org.apache.spark.sql.catalyst.DefaultParserDialect
|
||||
import org.apache.spark.sql.catalyst.analysis.EliminateSubQueries
|
||||
import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, EliminateSubQueries}
|
||||
import org.apache.spark.sql.catalyst.errors.DialectException
|
||||
import org.apache.spark.sql.execution.datasources.LogicalRelation
|
||||
import org.apache.spark.sql.hive.test.TestHive
|
||||
|
@ -138,6 +140,50 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils {
|
|||
(1 to 6).map(_ => Row("CA", 20151)))
|
||||
}
|
||||
|
||||
test("show functions") {
|
||||
val allFunctions =
|
||||
(FunctionRegistry.builtin.listFunction().toSet[String] ++
|
||||
org.apache.hadoop.hive.ql.exec.FunctionRegistry.getFunctionNames).toList.sorted
|
||||
checkAnswer(sql("SHOW functions"), allFunctions.map(Row(_)))
|
||||
checkAnswer(sql("SHOW functions abs"), Row("abs"))
|
||||
checkAnswer(sql("SHOW functions 'abs'"), Row("abs"))
|
||||
checkAnswer(sql("SHOW functions abc.abs"), Row("abs"))
|
||||
checkAnswer(sql("SHOW functions `abc`.`abs`"), Row("abs"))
|
||||
checkAnswer(sql("SHOW functions `abc`.`abs`"), Row("abs"))
|
||||
checkAnswer(sql("SHOW functions `~`"), Row("~"))
|
||||
checkAnswer(sql("SHOW functions `a function doens't exist`"), Nil)
|
||||
checkAnswer(sql("SHOW functions `weekofyea.*`"), Row("weekofyear"))
|
||||
// this probably will failed if we add more function with `sha` prefixing.
|
||||
checkAnswer(sql("SHOW functions `sha.*`"), Row("sha") :: Row("sha1") :: Row("sha2") :: Nil)
|
||||
}
|
||||
|
||||
test("describe functions") {
|
||||
// The Spark SQL built-in functions
|
||||
checkExistence(sql("describe function extended upper"), true,
|
||||
"Function: upper",
|
||||
"Class: org.apache.spark.sql.catalyst.expressions.Upper",
|
||||
"Usage: upper(str) - Returns str with all characters changed to uppercase",
|
||||
"Extended Usage:",
|
||||
"> SELECT upper('SparkSql')",
|
||||
"'SPARKSQL'")
|
||||
|
||||
checkExistence(sql("describe functioN Upper"), true,
|
||||
"Function: upper",
|
||||
"Class: org.apache.spark.sql.catalyst.expressions.Upper",
|
||||
"Usage: upper(str) - Returns str with all characters changed to uppercase")
|
||||
|
||||
checkExistence(sql("describe functioN Upper"), false,
|
||||
"Extended Usage")
|
||||
|
||||
checkExistence(sql("describe functioN abcadf"), true,
|
||||
"Function: abcadf is not found.")
|
||||
|
||||
checkExistence(sql("describe functioN `~`"), true,
|
||||
"Function: ~",
|
||||
"Class: org.apache.hadoop.hive.ql.udf.UDFOPBitNot",
|
||||
"Usage: ~ n - Bitwise not")
|
||||
}
|
||||
|
||||
test("SPARK-5371: union with null and sum") {
|
||||
val df = Seq((1, 1)).toDF("c1", "c2")
|
||||
df.registerTempTable("table1")
|
||||
|
|
Loading…
Reference in a new issue