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 d3b2491cd705..6fdc7f4a5819 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 @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, NoSuchFunctionException} import org.apache.spark.sql.catalyst.catalog.{CatalogFunction, FunctionResource} import org.apache.spark.sql.catalyst.expressions.{Attribute, ExpressionInfo} +import org.apache.spark.sql.catalyst.util.StringUtils import org.apache.spark.sql.types.{StringType, StructField, StructType} @@ -222,6 +223,21 @@ case class ShowFunctionsCommand( case (f, "USER") if showUserFunctions => f.unquotedString case (f, "SYSTEM") if showSystemFunctions => f.unquotedString } - functionNames.sorted.map(Row(_)) + // Hard code "<>", "!=", "between", and "case" for now as there is no corresponding functions. + // "<>", "!=", "between", and "case" is SystemFunctions, only show when showSystemFunctions=true + if (showSystemFunctions) { + (functionNames ++ + StringUtils.filterPattern(FunctionsCommand.virtualOperators, pattern.getOrElse("*"))) + .sorted.map(Row(_)) + } else { + functionNames.sorted.map(Row(_)) + } + } } + +object FunctionsCommand { + // operators that do not have corresponding functions. + // They should be handled `DescribeFunctionCommand`, `ShowFunctionsCommand` + val virtualOperators = Seq("!=", "<>", "between", "case") +} 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 1afe3976b2a1..630489ad9c60 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 @@ -31,6 +31,7 @@ import org.apache.spark.sql.catalyst.util.StringUtils import org.apache.spark.sql.execution.HiveResult.hiveResultString import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, SortAggregateExec} import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec +import org.apache.spark.sql.execution.command.FunctionsCommand import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan @@ -59,7 +60,8 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession { test("show functions") { def getFunctions(pattern: String): Seq[Row] = { StringUtils.filterPattern( - spark.sessionState.catalog.listFunctions("default").map(_._1.funcName), pattern) + spark.sessionState.catalog.listFunctions("default").map(_._1.funcName) + ++ FunctionsCommand.virtualOperators, pattern) .map(Row(_)) } 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 70b1db8e5f0d..6bd240431de8 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 @@ -2064,7 +2064,8 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { test("show functions") { withUserDefinedFunction("add_one" -> true) { - val numFunctions = FunctionRegistry.functionSet.size.toLong + val numFunctions = FunctionRegistry.functionSet.size.toLong + + FunctionsCommand.virtualOperators.size.toLong assert(sql("show functions").count() === numFunctions) assert(sql("show system functions").count() === numFunctions) assert(sql("show all functions").count() === numFunctions) 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 587eab4a2481..aa694ea274d7 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 @@ -32,6 +32,7 @@ import org.apache.hadoop.io.{LongWritable, Writable} import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.catalyst.plans.logical.Project +import org.apache.spark.sql.execution.command.FunctionsCommand import org.apache.spark.sql.functions.max import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf @@ -563,7 +564,8 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { checkAnswer( sql("SELECT testUDFToListInt(s) FROM inputTable"), Seq(Row(Seq(1, 2, 3)))) - assert(sql("show functions").count() == numFunc + 1) + assert(sql("show functions").count() == + numFunc + FunctionsCommand.virtualOperators.size + 1) assert(spark.catalog.listFunctions().count() == numFunc + 1) } } 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 736a2dcfad29..d2ca434bc211 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 @@ -33,7 +33,7 @@ import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, Functio import org.apache.spark.sql.catalyst.catalog.{CatalogTableType, CatalogUtils, HiveTableRelation} import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias} -import org.apache.spark.sql.execution.command.LoadDataCommand +import org.apache.spark.sql.execution.command.{FunctionsCommand, LoadDataCommand} import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.functions._ import org.apache.spark.sql.hive.{HiveExternalCatalog, HiveUtils} @@ -192,6 +192,11 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { allBuiltinFunctions.foreach { f => assert(allFunctions.contains(f)) } + + FunctionsCommand.virtualOperators.foreach { f => + assert(allFunctions.contains(f)) + } + withTempDatabase { db => def createFunction(names: Seq[String]): Unit = { names.foreach { name =>