This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new 91f95a7de9e [SPARK-39641][SQL][TESTS] Unify v1 and v2 SHOW FUNCTIONS 
tests
91f95a7de9e is described below

commit 91f95a7de9e524b9fd4730a3125c17622b67a760
Author: Max Gekk <max.g...@gmail.com>
AuthorDate: Thu Jun 30 22:03:30 2022 +0300

    [SPARK-39641][SQL][TESTS] Unify v1 and v2 SHOW FUNCTIONS tests
    
    ### What changes were proposed in this pull request?
    1. Move `SHOW FUNCTIONS` parsing tests to `ShowFunctionsParserSuite`.
    2. Put common v2 `SHOW FUNCTIONS` tests to `v2.ShowFunctionsSuite`.
    3. Refactor existing tests of temporary/permanent functions in v1 catalog, 
and place them to `v1.ShowTempFunctionsSuite` (temporary) and 
`hive.execution.command.ShowFunctionsSuite` (permanent).
    
    ### Why are the changes needed?
    1. The unification will allow to run common `SHOW FUNCTIONS` tests for both 
DSv1/Hive DSv1 and DSv2
    2. We can detect missing features and differences between DSv1 and DSv2 
implementations.
    
    ### Does this PR introduce _any_ user-facing change?
    No.
    
    ### How was this patch tested?
    By running the modified test suites:
    ```
    $ build/sbt "testOnly *DDLParserSuite"
    $ build/sbt "test:testOnly *DDLSuite"
    $ build/sbt "test:testOnly *DataSourceV2FunctionSuite"
    $ build/sbt "test:testOnly *SQLQuerySuite"
    ```
    and new test suites:
    ```
    $ build/sbt "sql/test:testOnly *ShowFunctionsParserSuite"
    $ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly 
*ShowFunctionsSuite"
    $ build/sbt "test:testOnly *ShowTempFunctionsSuite"
    ```
    
    Closes #37017 from MaxGekk/show-functions-v2.
    
    Lead-authored-by: Max Gekk <max.g...@gmail.com>
    Co-authored-by: Maxim Gekk <max.g...@gmail.com>
    Signed-off-by: Max Gekk <max.g...@gmail.com>
---
 .../spark/sql/catalyst/parser/DDLParserSuite.scala |  39 --------
 .../sql/connector/catalog/InMemoryCatalog.scala    |   4 +
 .../scala/org/apache/spark/sql/SQLQuerySuite.scala |  38 --------
 .../sql/connector/DataSourceV2FunctionSuite.scala  |   9 --
 .../spark/sql/execution/command/DDLSuite.scala     |  19 +---
 .../command/ShowFunctionsParserSuite.scala         |  68 +++++++++++++
 .../execution/command/ShowFunctionsSuiteBase.scala |  72 ++++++++++++++
 .../execution/command/v1/ShowFunctionsSuite.scala  | 107 +++++++++++++++++++++
 .../execution/command/v2/CommandSuiteBase.scala    |  19 +++-
 .../execution/command/v2/ShowFunctionsSuite.scala  |  40 ++++++++
 .../spark/sql/hive/execution/HiveUDFSuite.scala    |  27 ------
 .../spark/sql/hive/execution/SQLQuerySuite.scala   |  52 +---------
 .../execution/command/ShowFunctionsSuite.scala     |  71 ++++++++++++++
 13 files changed, 382 insertions(+), 183 deletions(-)

diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala
index 62195823bcc..f06377f36a8 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala
@@ -2042,45 +2042,6 @@ class DDLParserSuite extends AnalysisTest {
       DescribeFunction(createFuncPlan(Seq("a", "b", "c")), true))
   }
 
-  test("SHOW FUNCTIONS") {
-    val nsPlan = UnresolvedNamespace(Nil)
-    comparePlans(
-      parsePlan("SHOW FUNCTIONS"),
-      ShowFunctions(nsPlan, true, true, None))
-    comparePlans(
-      parsePlan("SHOW USER FUNCTIONS"),
-      ShowFunctions(nsPlan, true, false, None))
-    comparePlans(
-      parsePlan("SHOW user FUNCTIONS"),
-      ShowFunctions(nsPlan, true, false, None))
-    comparePlans(
-      parsePlan("SHOW SYSTEM FUNCTIONS"),
-      ShowFunctions(nsPlan, false, true, None))
-    comparePlans(
-      parsePlan("SHOW ALL FUNCTIONS"),
-      ShowFunctions(nsPlan, true, true, None))
-    comparePlans(
-      parsePlan("SHOW FUNCTIONS 'funct*'"),
-      ShowFunctions(nsPlan, true, true, Some("funct*")))
-    comparePlans(
-      parsePlan("SHOW FUNCTIONS LIKE 'funct*'"),
-      ShowFunctions(nsPlan, true, true, Some("funct*")))
-    comparePlans(
-      parsePlan("SHOW FUNCTIONS IN db LIKE 'funct*'"),
-      ShowFunctions(UnresolvedNamespace(Seq("db")), true, true, 
Some("funct*")))
-
-    // The legacy syntax.
-    comparePlans(
-      parsePlan("SHOW FUNCTIONS a"),
-      ShowFunctions(nsPlan, true, true, Some("a")))
-    comparePlans(
-      parsePlan("SHOW FUNCTIONS LIKE a"),
-      ShowFunctions(nsPlan, true, true, Some("a")))
-    comparePlans(
-      parsePlan("SHOW FUNCTIONS LIKE a.b.c"),
-      ShowFunctions(UnresolvedNamespace(Seq("a", "b")), true, true, Some("c")))
-  }
-
   test("REFRESH FUNCTION") {
     def createFuncPlan(name: Seq[String]): UnresolvedFunc = {
       UnresolvedFunc(name, "REFRESH FUNCTION", true, None)
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryCatalog.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryCatalog.scala
index be3baf92520..aa9914d760f 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryCatalog.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryCatalog.scala
@@ -56,6 +56,10 @@ class InMemoryCatalog extends InMemoryTableCatalog with 
FunctionCatalog {
     functions.put(ident, fn)
   }
 
+  def dropFunction(ident: Identifier): Unit = {
+    functions.remove(ident)
+  }
+
   def clearFunctions(): Unit = {
     functions.clear()
   }
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 056e59b3b86..edc8d3dae77 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
@@ -30,12 +30,10 @@ import org.apache.commons.io.FileUtils
 
 import org.apache.spark.{AccumulatorSuite, SparkException}
 import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart}
-import org.apache.spark.sql.catalyst.analysis.FunctionRegistry
 import org.apache.spark.sql.catalyst.expressions.{GenericRow, Hex}
 import org.apache.spark.sql.catalyst.expressions.aggregate.{Complete, Partial}
 import org.apache.spark.sql.catalyst.optimizer.{ConvertToLocalRelation, 
NestedColumnAliasingSuite}
 import org.apache.spark.sql.catalyst.plans.logical.{LocalLimit, Project, 
RepartitionByExpression, Sort}
-import org.apache.spark.sql.catalyst.util.StringUtils
 import org.apache.spark.sql.execution.{CommandResultExec, UnionExec}
 import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper
 import org.apache.spark.sql.execution.aggregate._
@@ -78,42 +76,6 @@ class SQLQuerySuite extends QueryTest with 
SharedSparkSession with AdaptiveSpark
     }
   }
 
-  test("show functions") {
-    def getFunctions(pattern: String): Seq[Row] = {
-      StringUtils.filterPattern(
-        spark.sessionState.catalog.listFunctions("default").map(_._1.funcName)
-        ++ FunctionRegistry.builtinOperators.keys, pattern)
-        .map(Row(_))
-    }
-
-    def createFunction(names: Seq[String]): Unit = {
-      names.foreach { name =>
-        spark.udf.register(name, (arg1: Int, arg2: String) => arg2 + arg1)
-      }
-    }
-
-    def dropFunction(names: Seq[String]): Unit = {
-      names.foreach { name =>
-        spark.sessionState.catalog.dropTempFunction(name, false)
-      }
-    }
-
-    val functions = Array("ilog", "logi", "logii", "logiii", "crc32i", 
"cubei", "cume_disti",
-      "isize", "ispace", "to_datei", "date_addi", "current_datei")
-
-    createFunction(functions)
-
-    checkAnswer(sql("SHOW functions"), getFunctions("*"))
-    assert(sql("SHOW functions").collect().size > 200)
-
-    Seq("^c*", "*e$", "log*", "*date*").foreach { pattern =>
-      // For the pattern part, only '*' and '|' are allowed as wildcards.
-      // For '*', we need to replace it to '.*'.
-      checkAnswer(sql(s"SHOW FUNCTIONS '$pattern'"), getFunctions(pattern))
-    }
-    dropFunction(functions)
-  }
-
   test("describe functions") {
     checkKeywordsExist(sql("describe function extended upper"),
       "Function: upper",
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2FunctionSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2FunctionSuite.scala
index 4a070ace377..8277a8e2979 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2FunctionSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2FunctionSuite.scala
@@ -168,15 +168,6 @@ class DataSourceV2FunctionSuite extends 
DatasourceV2SQLBase {
     assert(e1.message.contains("requires a single-part namespace"))
   }
 
-  test("SHOW FUNCTIONS: only support session catalog") {
-    addFunction(Identifier.of(Array.empty, "abc"), new JavaStrLen(new 
JavaStrLenNoImpl))
-
-    val e = intercept[AnalysisException] {
-      sql(s"SHOW FUNCTIONS LIKE testcat.abc")
-    }
-    assert(e.message.contains("Catalog testcat does not support functions"))
-  }
-
   test("DROP FUNCTION: only support session catalog") {
     addFunction(Identifier.of(Array.empty, "abc"), new JavaStrLen(new 
JavaStrLenNoImpl))
 
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 1921de9b8b4..f0925826b09 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
@@ -28,7 +28,7 @@ import org.apache.spark.{SparkException, SparkFiles}
 import org.apache.spark.internal.config
 import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode}
 import org.apache.spark.sql.catalyst.{FunctionIdentifier, QualifiedTableName, 
TableIdentifier}
-import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, 
TableFunctionRegistry, TempTableAlreadyExistsException}
+import org.apache.spark.sql.catalyst.analysis.TempTableAlreadyExistsException
 import org.apache.spark.sql.catalyst.catalog._
 import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
 import org.apache.spark.sql.connector.catalog.SupportsNamespaces.PROP_OWNER
@@ -1464,23 +1464,6 @@ abstract class DDLSuite extends QueryTest with 
SQLTestUtils {
     }
   }
 
-  test("show functions") {
-    withUserDefinedFunction("add_one" -> true) {
-      val numFunctions = FunctionRegistry.functionSet.size.toLong +
-        TableFunctionRegistry.functionSet.size.toLong +
-        FunctionRegistry.builtinOperators.size.toLong
-      assert(sql("show functions").count() === numFunctions)
-      assert(sql("show system functions").count() === numFunctions)
-      assert(sql("show all functions").count() === numFunctions)
-      assert(sql("show user functions").count() === 0L)
-      spark.udf.register("add_one", (x: Long) => x + 1)
-      assert(sql("show functions").count() === numFunctions + 1L)
-      assert(sql("show system functions").count() === numFunctions)
-      assert(sql("show all functions").count() === numFunctions + 1L)
-      assert(sql("show user functions").count() === 1L)
-    }
-  }
-
   test("show columns - negative test") {
     // When case sensitivity is true, the user supplied database name in table 
identifier
     // should match the supplied database name in case sensitive way.
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowFunctionsParserSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowFunctionsParserSuite.scala
new file mode 100644
index 00000000000..cd99d2f3cb0
--- /dev/null
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowFunctionsParserSuite.scala
@@ -0,0 +1,68 @@
+/*
+ * 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.execution.command
+
+import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, 
UnresolvedNamespace}
+import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan
+import org.apache.spark.sql.catalyst.plans.logical.ShowFunctions
+
+class ShowFunctionsParserSuite extends AnalysisTest {
+  val nsPlan = UnresolvedNamespace(Nil)
+
+  test("show functions in the scope") {
+    comparePlans(
+      parsePlan("SHOW FUNCTIONS"),
+      ShowFunctions(nsPlan, true, true, None))
+    comparePlans(
+      parsePlan("SHOW USER FUNCTIONS"),
+      ShowFunctions(nsPlan, true, false, None))
+    comparePlans(
+      parsePlan("SHOW user FUNCTIONS"),
+      ShowFunctions(nsPlan, true, false, None))
+    comparePlans(
+      parsePlan("SHOW SYSTEM FUNCTIONS"),
+      ShowFunctions(nsPlan, false, true, None))
+    comparePlans(
+      parsePlan("SHOW ALL FUNCTIONS"),
+      ShowFunctions(nsPlan, true, true, None))
+  }
+
+  test("show functions matched to a pattern") {
+    comparePlans(
+      parsePlan("SHOW FUNCTIONS 'funct*'"),
+      ShowFunctions(nsPlan, true, true, Some("funct*")))
+    comparePlans(
+      parsePlan("SHOW FUNCTIONS LIKE 'funct*'"),
+      ShowFunctions(nsPlan, true, true, Some("funct*")))
+    comparePlans(
+      parsePlan("SHOW FUNCTIONS IN db LIKE 'funct*'"),
+      ShowFunctions(UnresolvedNamespace(Seq("db")), true, true, 
Some("funct*")))
+  }
+
+  test("show functions using the legacy syntax") {
+    comparePlans(
+      parsePlan("SHOW FUNCTIONS a"),
+      ShowFunctions(nsPlan, true, true, Some("a")))
+    comparePlans(
+      parsePlan("SHOW FUNCTIONS LIKE a"),
+      ShowFunctions(nsPlan, true, true, Some("a")))
+    comparePlans(
+      parsePlan("SHOW FUNCTIONS LIKE a.b.c"),
+      ShowFunctions(UnresolvedNamespace(Seq("a", "b")), true, true, Some("c")))
+  }
+}
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowFunctionsSuiteBase.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowFunctionsSuiteBase.scala
new file mode 100644
index 00000000000..130914d35d9
--- /dev/null
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowFunctionsSuiteBase.scala
@@ -0,0 +1,72 @@
+/*
+ * 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.execution.command
+
+import java.util.Locale
+
+import org.apache.spark.sql.QueryTest
+import org.apache.spark.util.Utils
+
+/**
+ * This base suite contains unified tests for the `SHOW FUNCTIONS` command 
that check V1 and V2
+ * table catalogs. The tests that cannot run for all supported catalogs are 
located in more
+ * specific test suites:
+ *
+ *   - V2 catalog tests: 
`org.apache.spark.sql.execution.command.v2.ShowFunctionsSuite`
+ *   - V1 catalog tests:
+ *     `org.apache.spark.sql.execution.command.v1.ShowFunctionsSuiteBase`
+ *     - Temporary functions:
+ *        `org.apache.spark.sql.execution.command.v1.ShowTempFunctionsSuite`
+ *     - Permanent functions:
+ *        `org.apache.spark.sql.hive.execution.command.ShowFunctionsSuite`
+ */
+trait ShowFunctionsSuiteBase extends QueryTest with DDLCommandTestUtils {
+  override val command = "SHOW FUNCTIONS"
+
+  protected def createFunction(name: String): Unit = {}
+  protected def dropFunction(name: String): Unit = {}
+  protected def showFun(ns: String, name: String): String = 
s"$ns.$name".toLowerCase(Locale.ROOT)
+
+  /**
+   * Drops function `funName` after calling `f`.
+   */
+  protected def withFunction(functionNames: String*)(f: => Unit): Unit = {
+    Utils.tryWithSafeFinally(f) {
+      functionNames.foreach(dropFunction)
+    }
+  }
+
+  protected def withNamespaceAndFuns(ns: String, funNames: Seq[String], cat: 
String = catalog)
+      (f: (String, Seq[String]) => Unit): Unit = {
+    val nsCat = s"$cat.$ns"
+    withNamespace(nsCat) {
+      sql(s"CREATE NAMESPACE $nsCat")
+      val nsCatFns = funNames.map(funName => s"$nsCat.$funName")
+      withFunction(nsCatFns: _*) {
+        f(nsCat, nsCatFns)
+      }
+    }
+  }
+
+  protected def withNamespaceAndFun(ns: String, funName: String, cat: String = 
catalog)
+      (f: (String, String) => Unit): Unit = {
+    withNamespaceAndFuns(ns, Seq(funName), cat) { case (ns, Seq(name)) =>
+      f(ns, name)
+    }
+  }
+}
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowFunctionsSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowFunctionsSuite.scala
new file mode 100644
index 00000000000..f14b550688b
--- /dev/null
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowFunctionsSuite.scala
@@ -0,0 +1,107 @@
+/*
+ * 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.execution.command.v1
+
+import java.util.Locale
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.execution.command
+
+/**
+ * This base suite contains unified tests for the `SHOW FUNCTIONS` command 
that checks V1
+ * table catalogs. The tests that cannot run for all V1 catalogs are located 
in more
+ * specific test suites:
+ *
+ *   - Temporary functions of V1 catalog:
+ *     `org.apache.spark.sql.execution.command.v1.ShowTempFunctionsSuite`
+ *   - Permanent functions of V1 catalog:
+ *     `org.apache.spark.sql.hive.execution.command.ShowFunctionsSuite`
+ */
+trait ShowFunctionsSuiteBase extends command.ShowFunctionsSuiteBase
+  with command.TestsV1AndV2Commands {
+
+  test("show a function") {
+    withNamespaceAndFun("ns", "iiilog") { (ns, f) =>
+      val totalFuns = sql(s"SHOW FUNCTIONS IN $ns").count()
+      createFunction(f)
+      assert(sql(s"SHOW FUNCTIONS IN $ns").count() - totalFuns === 1)
+      assert(!sql(s"SHOW FUNCTIONS IN $ns").filter("contains(function, 
'iiilog')").isEmpty)
+    }
+  }
+
+  test("show a function in the USER name space") {
+    withNamespaceAndFun("ns", "logiii") { (ns, f) =>
+      assert(sql(s"SHOW USER FUNCTIONS IN $ns").count() === 0)
+      createFunction(f)
+      checkAnswer(sql(s"SHOW USER FUNCTIONS IN $ns"), Row(showFun("ns", 
"logiii")))
+    }
+  }
+
+  test("show functions in the SYSTEM name space") {
+    withNamespaceAndFun("ns", "date_addi") { (ns, f) =>
+      val systemFuns = sql(s"SHOW SYSTEM FUNCTIONS IN $ns").count()
+      assert(systemFuns > 0)
+      createFunction(f)
+      assert(sql(s"SHOW SYSTEM FUNCTIONS IN $ns").count() === systemFuns)
+    }
+  }
+
+  test("show functions among both user and system defined functions") {
+    withNamespaceAndFun("ns", "current_datei") { (ns, f) =>
+      val allFuns = sql(s"SHOW ALL FUNCTIONS IN $ns").collect()
+      assert(allFuns.nonEmpty)
+      createFunction(f)
+      checkAnswer(
+        sql(s"SHOW ALL FUNCTIONS IN $ns"),
+        allFuns :+ Row(showFun("ns", "current_datei")))
+    }
+  }
+
+  test("show functions matched to the wildcard pattern") {
+    val testFuns = Seq("crc32i", "crc16j", "date1900", "Date1")
+    withNamespaceAndFuns("ns", testFuns) { (ns, funs) =>
+      assert(sql(s"SHOW USER FUNCTIONS IN $ns").isEmpty)
+      funs.foreach(createFunction)
+      checkAnswer(
+        sql(s"SHOW USER FUNCTIONS IN $ns LIKE '*'"),
+        testFuns.map(testFun => Row(showFun("ns", testFun))))
+      checkAnswer(
+        sql(s"SHOW USER FUNCTIONS IN $ns LIKE '*rc*'"),
+        Seq("crc32i", "crc16j").map(testFun => Row(showFun("ns", testFun))))
+    }
+  }
+}
+
+/**
+ * The class contains tests for the `SHOW FUNCTIONS` command to check 
temporary functions.
+ */
+class ShowTempFunctionsSuite extends ShowFunctionsSuiteBase with 
CommandSuiteBase {
+  override def commandVersion: String = 
super[ShowFunctionsSuiteBase].commandVersion
+
+  override protected def createFunction(name: String): Unit = {
+    spark.udf.register(name, (arg1: Int, arg2: String) => arg2 + arg1)
+  }
+
+  override protected def dropFunction(name: String): Unit = {
+    spark.sessionState.catalog.dropTempFunction(name, false)
+  }
+
+  override protected def showFun(ns: String, name: String): String = {
+    s"$catalog.$ns.$name".toLowerCase(Locale.ROOT)
+  }
+}
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CommandSuiteBase.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CommandSuiteBase.scala
index 1ff9e74f180..ac38a589ffd 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CommandSuiteBase.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CommandSuiteBase.scala
@@ -20,8 +20,10 @@ package org.apache.spark.sql.execution.command.v2
 import org.apache.spark.SparkConf
 import org.apache.spark.sql.catalyst.analysis.ResolvePartitionSpec
 import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
-import org.apache.spark.sql.connector.catalog.{CatalogV2Implicits, Identifier, 
InMemoryPartitionTable, InMemoryPartitionTableCatalog, InMemoryTableCatalog}
+import org.apache.spark.sql.connector.catalog.{CatalogV2Implicits, Identifier, 
InMemoryCatalog, InMemoryPartitionTable, InMemoryPartitionTableCatalog, 
InMemoryTableCatalog}
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction
 import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.util.Utils
 
 /**
  * The trait contains settings and utility functions. It can be mixed to the 
test suites for
@@ -33,12 +35,14 @@ trait CommandSuiteBase extends SharedSparkSession {
   def catalogVersion: String = "V2" // The catalog version is added to test 
names
   def commandVersion: String = "V2" // The command version is added to test 
names
   def catalog: String = "test_catalog" // The default V2 catalog for testing
+  def funCatalog: String = s"fun_$catalog"
   def defaultUsing: String = "USING _" // The clause is used in creating v2 
tables under testing
 
   // V2 catalogs created and used especially for testing
   override def sparkConf: SparkConf = super.sparkConf
     .set(s"spark.sql.catalog.$catalog", 
classOf[InMemoryPartitionTableCatalog].getName)
     .set(s"spark.sql.catalog.non_part_$catalog", 
classOf[InMemoryTableCatalog].getName)
+    .set(s"spark.sql.catalog.$funCatalog", classOf[InMemoryCatalog].getName)
 
   def checkLocation(
       t: String,
@@ -61,4 +65,17 @@ trait CommandSuiteBase extends SharedSparkSession {
     assert(partMetadata.containsKey("location"))
     assert(partMetadata.get("location") === expected)
   }
+
+
+  def withFun(ident: Identifier, fn: UnboundFunction)(f: => Unit): Unit = {
+    val cat = spark.sessionState
+      .catalogManager
+      .catalog(funCatalog)
+      .asInstanceOf[InMemoryCatalog]
+
+    cat.createFunction(ident, fn)
+    Utils.tryWithSafeFinally(f) {
+      cat.dropFunction(ident)
+    }
+  }
 }
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowFunctionsSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowFunctionsSuite.scala
new file mode 100644
index 00000000000..88c16a5552a
--- /dev/null
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowFunctionsSuite.scala
@@ -0,0 +1,40 @@
+/*
+ * 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.execution.command.v2
+
+import test.org.apache.spark.sql.connector.catalog.functions.JavaStrLen
+import 
test.org.apache.spark.sql.connector.catalog.functions.JavaStrLen.JavaStrLenNoImpl
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.connector.catalog.Identifier
+import org.apache.spark.sql.execution.command
+
+/**
+ * The class contains tests for the `SHOW FUNCTIONS` command to check V2 table 
catalogs.
+ */
+class ShowFunctionsSuite extends command.ShowFunctionsSuiteBase with 
CommandSuiteBase {
+
+  test("only support session catalog") {
+    withFun(Identifier.of(Array.empty, "abc"), new JavaStrLen(new 
JavaStrLenNoImpl)) {
+      val e = intercept[AnalysisException] {
+        sql(s"SHOW FUNCTIONS LIKE $funCatalog.abc")
+      }
+      assert(e.getMessage === s"Catalog $funCatalog does not support 
functions")
+    }
+  }
+}
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 fc2501c117d..084850e6f0c 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
@@ -34,7 +34,6 @@ import org.apache.hadoop.io.{LongWritable, Writable}
 
 import org.apache.spark.{SparkFiles, TestUtils}
 import org.apache.spark.sql.{AnalysisException, QueryTest, Row}
-import org.apache.spark.sql.catalyst.analysis.FunctionRegistry
 import org.apache.spark.sql.catalyst.plans.logical.Project
 import org.apache.spark.sql.functions.max
 import org.apache.spark.sql.hive.test.TestHiveSingleton
@@ -551,32 +550,6 @@ class HiveUDFSuite extends QueryTest with 
TestHiveSingleton with SQLTestUtils {
     }
   }
 
-  test("Show persistent functions") {
-    val testData = spark.sparkContext.parallelize(StringCaseClass("") :: 
Nil).toDF()
-    withTempView("inputTable") {
-      testData.createOrReplaceTempView("inputTable")
-      withUserDefinedFunction("testUDFToListInt" -> false) {
-        val numFunc = spark.catalog.listFunctions().count()
-        sql(s"CREATE FUNCTION testUDFToListInt AS 
'${classOf[UDFToListInt].getName}'")
-        assert(spark.catalog.listFunctions().count() == numFunc + 1)
-        checkAnswer(
-          sql("SELECT testUDFToListInt(s) FROM inputTable"),
-          Seq(Row(Seq(1, 2, 3))))
-        assert(sql("show functions").count() ==
-          numFunc + FunctionRegistry.builtinOperators.size + 1)
-        assert(spark.catalog.listFunctions().count() == numFunc + 1)
-
-        withDatabase("db2") {
-          sql("CREATE DATABASE db2")
-          sql(s"CREATE FUNCTION db2.testUDFToListInt AS 
'${classOf[UDFToListInt].getName}'")
-          checkAnswer(
-            sql("SHOW FUNCTIONS IN db2 LIKE 'testUDF*'"),
-            Seq(Row("db2.testudftolistint")))
-        }
-      }
-    }
-  }
-
   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}'")
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 f2711db8399..9ce48a9ab75 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
@@ -29,7 +29,7 @@ import org.apache.hadoop.fs.{FileSystem, Path}
 import org.apache.spark.{SparkException, TestUtils}
 import org.apache.spark.sql._
 import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, 
FunctionRegistry}
+import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases
 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}
@@ -195,56 +195,6 @@ abstract class SQLQuerySuiteBase extends QueryTest with 
SQLTestUtils with TestHi
     }
   }
 
-  test("show functions") {
-    val allBuiltinFunctions = 
FunctionRegistry.builtin.listFunction().map(_.unquotedString)
-    val allFunctions = sql("SHOW functions").collect().map(r => r(0))
-    allBuiltinFunctions.foreach { f =>
-      assert(allFunctions.contains(f))
-    }
-
-    FunctionRegistry.builtinOperators.keys.foreach { f =>
-      assert(allFunctions.contains(f))
-    }
-
-    withTempDatabase { db =>
-      def createFunction(names: Seq[String]): Unit = {
-        names.foreach { name =>
-          sql(
-            s"""
-              |CREATE TEMPORARY FUNCTION $name
-              |AS '${classOf[PairUDF].getName}'
-            """.stripMargin)
-        }
-      }
-      def dropFunction(names: Seq[String]): Unit = {
-        names.foreach { name =>
-          sql(s"DROP TEMPORARY FUNCTION $name")
-        }
-      }
-      createFunction(Seq("temp_abs", "temp_weekofyear", "temp_sha", 
"temp_sha1", "temp_sha2"))
-
-      checkAnswer(sql("SHOW functions temp_abs"), Row("temp_abs"))
-      checkAnswer(sql("SHOW functions 'temp_abs'"), Row("temp_abs"))
-      checkAnswer(sql(s"SHOW functions $db.temp_abs"), Row("temp_abs"))
-      checkAnswer(sql(s"SHOW functions `$db`.`temp_abs`"), Row("temp_abs"))
-      checkAnswer(sql(s"SHOW functions `$db`.`temp_abs`"), Row("temp_abs"))
-      checkAnswer(sql("SHOW functions `a function doesn't exist`"), Nil)
-      checkAnswer(sql("SHOW functions `temp_weekofyea*`"), 
Row("temp_weekofyear"))
-
-      // this probably will failed if we add more function with `sha` 
prefixing.
-      checkAnswer(
-        sql("SHOW functions `temp_sha*`"),
-        List(Row("temp_sha"), Row("temp_sha1"), Row("temp_sha2")))
-
-      // Test '|' for alternation.
-      checkAnswer(
-        sql("SHOW functions 'temp_sha*|temp_weekofyea*'"),
-        List(Row("temp_sha"), Row("temp_sha1"), Row("temp_sha2"), 
Row("temp_weekofyear")))
-
-      dropFunction(Seq("temp_abs", "temp_weekofyear", "temp_sha", "temp_sha1", 
"temp_sha2"))
-    }
-  }
-
   test("describe functions - built-in functions") {
     checkKeywordsExist(sql("describe function extended upper"),
       "Function: upper",
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowFunctionsSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowFunctionsSuite.scala
new file mode 100644
index 00000000000..9d40f33f404
--- /dev/null
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowFunctionsSuite.scala
@@ -0,0 +1,71 @@
+/*
+ * 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.hive.execution.command
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.execution.command.v1
+import org.apache.spark.sql.hive.execution.UDFToListInt
+
+/**
+ * The class contains tests for the `SHOW FUNCTIONS` command to check 
permanent functions.
+ */
+class ShowFunctionsSuite extends v1.ShowFunctionsSuiteBase with 
CommandSuiteBase {
+  override def commandVersion: String = 
super[ShowFunctionsSuiteBase].commandVersion
+
+  override protected def createFunction(name: String): Unit = {
+    sql(s"CREATE FUNCTION $name AS '${classOf[UDFToListInt].getName}'")
+  }
+  override protected def dropFunction(name: String): Unit = {
+    sql(s"DROP FUNCTION IF EXISTS $name")
+  }
+
+  test("show a function by its string name") {
+    val testFuns = Seq("crc32i", "crc16j")
+    withNamespaceAndFuns("ns", testFuns) { (ns, funs) =>
+      assert(sql(s"SHOW USER FUNCTIONS IN $ns").isEmpty)
+      funs.foreach(createFunction)
+      checkAnswer(
+        sql(s"SHOW USER FUNCTIONS IN $ns 'crc32i'"),
+        Row(showFun("ns", "crc32i")))
+    }
+  }
+
+  test("show functions matched to the '|' pattern") {
+    val testFuns = Seq("crc32i", "crc16j", "date1900", "Date1")
+    withNamespaceAndFuns("ns", testFuns) { (ns, funs) =>
+      assert(sql(s"SHOW USER FUNCTIONS IN $ns").isEmpty)
+      funs.foreach(createFunction)
+      checkAnswer(
+        sql(s"SHOW USER FUNCTIONS IN $ns LIKE 'crc32i|date1900'"),
+        Seq("crc32i", "date1900").map(testFun => Row(showFun("ns", testFun))))
+      checkAnswer(
+        sql(s"SHOW USER FUNCTIONS IN $ns LIKE 'crc32i|date*'"),
+        Seq("crc32i", "date1900", "Date1").map(testFun => Row(showFun("ns", 
testFun))))
+    }
+  }
+
+  test("show a function by its id") {
+    withNamespaceAndFun("ns", "crc32i") { (ns, fun) =>
+      assert(sql(s"SHOW USER FUNCTIONS IN $ns").isEmpty)
+      createFunction(fun)
+      checkAnswer(
+        sql(s"SHOW USER FUNCTIONS $fun"),
+        Row(showFun("ns", "crc32i")))
+    }
+  }
+}


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

Reply via email to