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

cloud-fan pushed a commit to branch branch-4.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-4.2 by this push:
     new e42a8eec7a07 [SPARK-56883][SQL] DESCRIBE FUNCTION for SQL UDFs
e42a8eec7a07 is described below

commit e42a8eec7a07c7908ebf9f412fccedde7b70ac03
Author: Serge Rielau <[email protected]>
AuthorDate: Tue May 19 09:16:45 2026 +0800

    [SPARK-56883][SQL] DESCRIBE FUNCTION for SQL UDFs
    
    ### What changes were proposed in this pull request?
    
    Renders a structured `DESCRIBE FUNCTION [EXTENDED]` output for SQL 
user-defined functions (temporary and persistent) in place of the generic 
`Function / Class / Usage:<json blob>` dump that `DescribeFunctionCommand` 
produces today for any function whose `ExpressionInfo.className != null`.
    
    For SQL UDFs the output becomes:
    
    - `Function:` qualified name
    - `Type:` `SCALAR` or `TABLE`
    - `Input:` parameter list (name + SQL type, column-aligned; `DEFAULT 
<expr>` and `'comment'` annotations are added in EXTENDED mode)
    - `Returns:` scalar return type, or the table return columns (column 
comments and defaults are added in EXTENDED mode)
    - EXTENDED only: `Comment`, `Collation`, `Deterministic`, `Data Access` 
(`CONTAINS SQL` / `READS SQL DATA`), `Configs`, `Owner`, `Create Time`, `Body`, 
and `SQL Path`.
    
    `SQL Path:` is emitted only when both `spark.sql.path.enabled = true` and a 
frozen path was persisted on the function at `CREATE FUNCTION` time 
(SPARK-56639 / SPARK-56520). The path is read from the function's 
`function.resolutionPath` property and rendered through 
`SqlPathFormat.formatForDisplay`, producing the same `` `catalog`.`namespace` 
`` format used elsewhere in DESCRIBE output. This shows the resolution path 
that the function will use during analysis — the creator's PATH froze [...]
    
    Behavior for builtin functions and non-SQL UDFs is unchanged.
    
    Class hierarchy / dispatch:
    
    - `SQLFunction` (catalyst): adds the `SCALAR` / `TABLE` constants and a new 
`fromExpressionInfo(info, parser)` constructor that reconstructs a 
`SQLFunction` from the JSON usage blob produced by `toExpressionInfo`. This is 
the same path used by both temp UDFs (which are not in the catalog) and 
persistent UDFs.
    - `DescribeFunctionCommand` (sql/core): when 
`SQLFunction.isSQLFunction(info.getClassName)` is true, dispatches to a new 
`describeSQLFunction(info, parser)` helper that emits the column-aligned 
key/value rows shown above. The frozen SQL PATH is rendered inline through 
`SqlPathFormat`; the temporary `DescribeFunctionCommandUtils` helper introduced 
for that purpose by SPARK-56639 is removed (its single responsibility is now 
absorbed by `describeSQLFunction`).
    - `SessionCatalog.registerFunction`: when a persistent SQL UDF is invoked 
for the first time, the function registry caches it. Previously the cached 
`ExpressionInfo` was always built via `makeExprInfoForHiveFunction`, which sets 
`usage = null`. That worked for the pre-existing `DESCRIBE FUNCTION` codepath 
(which doesn't read `usage`), but breaks the new `describeSQLFunction` path: 
after a SQL UDF has been invoked once, `DESCRIBE FUNCTION` reads back the 
cached info and `SQLFunction.fr [...]
    
    ### Why are the changes needed?
    
    `DESCRIBE FUNCTION` is intended to give users a human-readable description 
of a routine, analogous to `DESCRIBE TABLE` for tables. For SQL UDFs the 
current output instead exposes the internal serialization format:
    
    ```
    > DESCRIBE FUNCTION EXTENDED area;
     Function: default.area
     Class: sqlFunction.
     Usage: {"sqlFunction.inputParam":"width DOUBLE,height 
DOUBLE","sqlFunction.returnType":"DOUBLE","sqlFunction.expression":"width * 
height","sqlFunction.isTableFunc":"false",...}
     Extended Usage:
    ```
    
    That JSON blob is not part of any public surface, and the literal string 
`sqlFunction.` for `Class:` is meaningless to users. All of the structured 
metadata we need — signature, return type, body, characteristics, frozen SQL 
PATH — is already serialized in `ExpressionInfo`; this PR just formats it.
    
    ### Does this PR introduce _any_ user-facing change?
    
    Yes — the rows returned by `DESCRIBE FUNCTION [EXTENDED] <sql_udf>` change.
    
    Before:
    
    ```
    > DESCRIBE FUNCTION EXTENDED area;
     Function: default.area
     Class: sqlFunction.
     Usage: {"sqlFunction.inputParam":"width DOUBLE,height DOUBLE", ...}
     Extended Usage:
    ```
    
    After (simple case):
    
    ```
    > DESCRIBE FUNCTION EXTENDED area;
     Function:      default.area
     Type:          SCALAR
     Input:         width  DOUBLE 'width'
                    height DOUBLE 'height'
     Returns:       DOUBLE
     Comment:       compute area
     Deterministic: true
     Data Access:   CONTAINS SQL
     Owner:         <owner>
     Create Time:   <timestamp>
     Body:          width * height
    ```
    
    After (function created under `spark.sql.path.enabled = true` with a 
non-default PATH at CREATE time):
    
    ```
    > SET spark.sql.path.enabled = true;
    > SET PATH = spark_catalog.path_func_db_a, system.builtin;
    > CREATE FUNCTION frozen_fn() RETURNS INT RETURN (SELECT MAX(id) FROM 
frozen_t);
    > SET PATH = spark_catalog.path_func_db_b, system.builtin;
    > DESCRIBE FUNCTION EXTENDED default.frozen_fn;
     Function:      default.frozen_fn
     Type:          SCALAR
     Input:         ()
     Returns:       INT
     ...
     Body:          (SELECT MAX(id) FROM frozen_t)
     SQL Path:      `spark_catalog`.`path_func_db_a`, `system`.`builtin`
    ```
    
    `SQL Path` reflects the creator's frozen PATH, not the session's current 
`PATH` at describe time. Output for builtin functions, Hive UDFs, and other 
non-SQL UDFs is unchanged.
    
    ### How was this patch tested?
    
    Added four unit tests to `SQLFunctionSuite` (sql/core):
    
    - `describe SQL scalar functions` — temporary and persistent scalar UDFs 
with comments, defaults, and `EXTENDED` mode. Asserts `Function`, `Type`, 
`Input` (column-aligned, with `DEFAULT` and `'comment'` in extended mode), 
`Returns`, `Deterministic`, `Data Access`, `Comment`, `Create Time`, `Body`.
    - `describe SQL table functions` — table UDFs with explicit return columns; 
asserts `Type: TABLE`, `Returns` columns, and the EXTENDED-only fields.
    - `describe SQL functions with derived routine characteristics` — checks 
that `Deterministic` and `Data Access` reflect derived values for functions 
that read tables / call non-deterministic builtins, and that user-supplied 
characteristics are preserved.
    - The existing `SPARK-56639: SQL function uses frozen SQL path` test is 
extended: after switching `PATH` to a different namespace it invokes 
`default.frozen_fn` (populating the function-registry cache) and then runs 
`DESCRIBE FUNCTION EXTENDED default.frozen_fn`, asserting the `SQL Path:` row 
shows the *creator's* frozen path (`` `spark_catalog`.`path_func_db_a`, 
`system`.`builtin` ``) and does *not* mention the invoker's current path 
namespace. This extension also exercises the `Sess [...]
    
    Each describe test uses `checkKeywordsExist` against `DESCRIBE FUNCTION 
[EXTENDED] <name>` output.
    
    ### Was this patch authored or co-authored using generative AI tooling?
    
    Generated-by: Claude (claude-opus-4-7)
    
    Closes #55915 from srielau/SPARK-56883-describe-sql-udf.
    
    Authored-by: Serge Rielau <[email protected]>
    Signed-off-by: Wenchen Fan <[email protected]>
    (cherry picked from commit cfed631e5a093b47621a856a5b15d0224ccdb24d)
    Signed-off-by: Wenchen Fan <[email protected]>
---
 .../spark/sql/catalyst/catalog/SQLFunction.scala   |  74 +++++++++---
 .../sql/catalyst/catalog/SessionCatalog.scala      |  29 ++++-
 .../command/DescribeFunctionCommandUtils.scala     |  89 --------------
 .../spark/sql/execution/command/functions.scala    | 133 +++++++++++++++++----
 .../spark/sql/execution/SQLFunctionSuite.scala     | 133 +++++++++++++++++++++
 5 files changed, 324 insertions(+), 134 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SQLFunction.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SQLFunction.scala
index 07ca0a871248..25ce823337ef 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SQLFunction.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SQLFunction.scala
@@ -122,7 +122,10 @@ case class SQLFunction(
    * Convert the SQL function to a [[CatalogFunction]].
    */
   def toCatalogFunction: CatalogFunction = {
-    val props = sqlFunctionToProps ++ properties
+    // Persist function metadata (owner, createTime) alongside the SQL function
+    // body so the values survive a session restart and can be rendered by
+    // DESCRIBE FUNCTION EXTENDED.
+    val props = sqlFunctionToProps ++ functionMetadataToProps ++ properties
     CatalogFunction(
       identifier = name,
       className = SQL_FUNCTION_PREFIX,
@@ -187,6 +190,9 @@ case class SQLFunction(
 
 object SQLFunction {
 
+  val SCALAR = "SCALAR"
+  val TABLE = "TABLE"
+
   /**
    * Persisted frozen PATH for SQL function bodies when created with 
[[SQLConf.PATH_ENABLED]].
    * Serialized as a JSON array of path entries (same format as
@@ -227,21 +233,7 @@ object SQLFunction {
       }
       val blob = parts.sortBy(_._1).map(_._2).mkString
       val props = mapper.readValue(blob, classOf[Map[String, String]])
-      val isTableFunc = props(IS_TABLE_FUNC).toBoolean
-      val collation = props.get(COLLATION)
-      val returnType = parseReturnTypeText(props(RETURN_TYPE), isTableFunc, 
parser, collation)
-      SQLFunction(
-        name = function.identifier,
-        inputParam = props.get(INPUT_PARAM).map(parseRoutineParam(_, parser, 
collation)),
-        returnType = returnType.get,
-        exprText = props.get(EXPRESSION),
-        queryText = props.get(QUERY),
-        comment = props.get(COMMENT),
-        collation = collation,
-        deterministic = props.get(DETERMINISTIC).map(_.toBoolean),
-        containsSQL = props.get(CONTAINS_SQL).map(_.toBoolean),
-        isTableFunc = isTableFunc,
-        props.filterNot(_._1.startsWith(SQL_FUNCTION_PREFIX)))
+      fromProps(props, function.identifier, parser)
     } catch {
       case e: Exception =>
         throw new AnalysisException(
@@ -253,6 +245,56 @@ object SQLFunction {
     }
   }
 
+  /**
+   * Convert an [[ExpressionInfo]] into a SQL function.
+   */
+  def fromExpressionInfo(info: ExpressionInfo, parser: ParserInterface): 
SQLFunction = {
+    try {
+      val props = mapper.readValue(info.getUsage, classOf[Map[String, String]])
+      fromProps(props, FunctionIdentifier(info.getName, Option(info.getDb)), 
parser)
+    } catch {
+      case e: Exception =>
+        throw new AnalysisException(
+          errorClass = "CORRUPTED_CATALOG_FUNCTION",
+          messageParameters = Map(
+            "identifier" -> s"${info.getDb}.${info.getName}",
+            "className" -> s"${info.getClassName}"), cause = Some(e)
+        )
+    }
+  }
+
+  /**
+   * Build a [[SQLFunction]] from a deserialized property map and a function 
identifier.
+   * Shared by both [[fromCatalogFunction]] and [[fromExpressionInfo]] so all 
readers
+   * stay in sync as new properties are added.
+   *
+   * `OWNER` is optional and defaults to `None` when missing; `CREATE_TIME` 
falls back
+   * to the current wall-clock time so functions persisted before metadata was 
added
+   * to the catalog payload still load.
+   */
+  private def fromProps(
+      props: Map[String, String],
+      identifier: FunctionIdentifier,
+      parser: ParserInterface): SQLFunction = {
+    val isTableFunc = props(IS_TABLE_FUNC).toBoolean
+    val collation = props.get(COLLATION)
+    val returnType = parseReturnTypeText(props(RETURN_TYPE), isTableFunc, 
parser, collation)
+    SQLFunction(
+      name = identifier,
+      inputParam = props.get(INPUT_PARAM).map(parseRoutineParam(_, parser, 
collation)),
+      returnType = returnType.get,
+      exprText = props.get(EXPRESSION),
+      queryText = props.get(QUERY),
+      comment = props.get(COMMENT),
+      collation = collation,
+      deterministic = props.get(DETERMINISTIC).map(_.toBoolean),
+      containsSQL = props.get(CONTAINS_SQL).map(_.toBoolean),
+      isTableFunc = isTableFunc,
+      properties = props.filterNot(_._1.startsWith(SQL_FUNCTION_PREFIX)),
+      owner = props.get(OWNER),
+      createTimeMs = 
props.get(CREATE_TIME).map(_.toLong).getOrElse(System.currentTimeMillis))
+  }
+
   def parseDefault(text: String, parser: ParserInterface): Expression = {
     parser.parseExpression(text)
   }
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 32aa8cccbd93..9c863a7b55fe 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
@@ -2059,14 +2059,15 @@ class SessionCatalog(
       overrideIfExists: Boolean,
       functionBuilder: Option[FunctionBuilder] = None): Unit = {
     val builder = 
functionBuilder.getOrElse(makeFunctionBuilder(funcDefinition))
-    registerFunction(funcDefinition, overrideIfExists, functionRegistry, 
builder)
+    registerFunction(funcDefinition, overrideIfExists, functionRegistry, 
builder, info = None)
   }
 
   private def registerFunction[T](
       funcDefinition: CatalogFunction,
       overrideIfExists: Boolean,
       registry: FunctionRegistryBase[T],
-      functionBuilder: FunctionRegistryBase[T]#FunctionBuilder): Unit = {
+      functionBuilder: FunctionRegistryBase[T]#FunctionBuilder,
+      info: Option[ExpressionInfo]): Unit = {
     val func = funcDefinition.identifier
 
     // Determine the key to use for registration:
@@ -2098,8 +2099,18 @@ class SessionCatalog(
     if (registry.functionExists(identToRegister) && !overrideIfExists) {
       throw QueryCompilationErrors.functionAlreadyExistsError(func)
     }
-    val info = makeExprInfoForHiveFunction(funcDefinition)
-    registry.registerFunction(identToRegister, info, functionBuilder)
+    // Prefer caller-supplied info (the freshly-registered SQL UDF path passes 
a
+    // structured ExpressionInfo). Otherwise reconstruct one: SQL UDFs need the
+    // structured `usage` blob so DESCRIBE FUNCTION can rehydrate them; 
hive-style
+    // functions get the legacy info with `usage = null`.
+    val resolvedInfo = info.getOrElse {
+      if (funcDefinition.isUserDefinedFunction) {
+        UserDefinedFunction.fromCatalogFunction(funcDefinition, 
parser).toExpressionInfo
+      } else {
+        makeExprInfoForHiveFunction(funcDefinition)
+      }
+    }
+    registry.registerFunction(identToRegister, resolvedInfo, functionBuilder)
   }
 
   private def makeExprInfoForHiveFunction(func: CatalogFunction): 
ExpressionInfo = {
@@ -2230,11 +2241,16 @@ class SessionCatalog(
       val info = function.toExpressionInfo
       registry.registerFunction(tempIdentifier, info, functionBuilder)
     } else {
+      // We already have the UserDefinedFunction in hand, so skip the
+      // CatalogFunction -> ExpressionInfo round trip inside `registerFunction`
+      // and pass the structured ExpressionInfo (with owner/createTime 
preserved
+      // at CREATE-time values) directly to the registry.
       registerFunction(
         function.toCatalogFunction,
         overrideIfExists,
         registry,
-        functionBuilder)
+        functionBuilder,
+        info = Some(function.toExpressionInfo))
     }
   }
 
@@ -2590,7 +2606,8 @@ class SessionCatalog(
                 funcMetadata,
                 overrideIfExists = false,
                 functionRegistry,
-                makeFunctionBuilder(funcMetadata))
+                makeFunctionBuilder(funcMetadata),
+                info = None)
             }
             functionRegistry.lookupFunctionBuilder(qualifiedIdent).get
           }
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DescribeFunctionCommandUtils.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DescribeFunctionCommandUtils.scala
deleted file mode 100644
index 24b04a9e3faf..000000000000
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DescribeFunctionCommandUtils.scala
+++ /dev/null
@@ -1,89 +0,0 @@
-/*
- * 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
-
-import org.apache.spark.sql.SparkSession
-import org.apache.spark.sql.catalyst.FunctionIdentifier
-import org.apache.spark.sql.catalyst.catalog.{SQLFunction, SqlPathFormat, 
UserDefinedFunction}
-import org.apache.spark.sql.catalyst.expressions.ExpressionInfo
-
-/**
- * Helpers for [[DescribeFunctionCommand]] to retrieve and format
- * the frozen SQL PATH stored in SQL function metadata.
- */
-private[command] object DescribeFunctionCommandUtils {
-
-  /**
-   * Returns the frozen SQL PATH persisted for a SQL function, formatted
-   * for display. Persistent functions: loads [[CatalogFunction]] metadata
-   * from the catalog. Temporary SQL UDFs (not in catalog): falls back to
-   * parsing the usage JSON blob produced by [[SQLFunction.toExpressionInfo]].
-   */
-  private[command] def storedResolutionPathString(
-      sparkSession: SparkSession,
-      identifier: FunctionIdentifier,
-      info: ExpressionInfo): Option[String] = {
-    val rawJson = try {
-      val meta = sparkSession.sessionState.catalog
-        .getFunctionMetadata(identifier)
-      if (meta.isUserDefinedFunction) {
-        val udf = UserDefinedFunction.fromCatalogFunction(
-          meta,
-          sparkSession.sessionState.sqlParser)
-        udf.asInstanceOf[SQLFunction].functionStoredResolutionPath
-      } else {
-        None
-      }
-    } catch {
-      case _: org.apache.spark.sql.catalyst.analysis
-        .NoSuchFunctionException |
-          _: org.apache.spark.sql.catalyst.analysis
-            .NoSuchDatabaseException =>
-        extractResolutionPathFromSqlUdfUsage(info.getUsage)
-    }
-    rawJson.flatMap(formatStoredPath)
-  }
-
-  private def formatStoredPath(pathStr: String): Option[String] = {
-    SqlPathFormat.toDescribeJson(pathStr)
-      .flatMap(SqlPathFormat.formatForDisplay)
-  }
-
-  /**
-   * For temporary SQL UDFs not in the catalog, the resolution path may
-   * be embedded in the ExpressionInfo usage JSON blob. Returns None if
-   * the usage string is not JSON or does not contain the path key.
-   */
-  private def extractResolutionPathFromSqlUdfUsage(
-      usage: String): Option[String] = {
-    if (usage == null || usage.isEmpty) return None
-    try {
-      val map = UserDefinedFunction.mapper.readValue(
-        usage, classOf[util.HashMap[String, String]])
-      Option(map.get(SQLFunction.FUNCTION_RESOLUTION_PATH))
-        .filter(_.nonEmpty)
-    } catch {
-      case e: com.fasterxml.jackson.core.JsonProcessingException =>
-        throw new org.apache.spark.SparkException(
-          s"Corrupted SQL UDF metadata: expected JSON usage blob " +
-          s"but failed to parse: ${e.getMessage}", e)
-    }
-  }
-}
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 5929e5c56f90..9839a3edbbab 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
@@ -17,16 +17,20 @@
 
 package org.apache.spark.sql.execution.command
 
+import scala.collection.mutable.ArrayBuffer
+
 import org.apache.spark.sql.{Row, SparkSession}
 import org.apache.spark.sql.catalyst.FunctionIdentifier
 import org.apache.spark.sql.catalyst.analysis.FunctionRegistry
-import org.apache.spark.sql.catalyst.catalog.{CatalogFunction, 
FunctionResource, SQLFunction}
+import org.apache.spark.sql.catalyst.catalog.{CatalogFunction, 
FunctionResource, SQLFunction, SqlPathFormat}
 import org.apache.spark.sql.catalyst.expressions.{Attribute, ExpressionInfo}
+import org.apache.spark.sql.catalyst.parser.ParserInterface
 import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes
 import org.apache.spark.sql.catalyst.util.StringUtils
 import org.apache.spark.sql.connector.catalog.CatalogManager
 import org.apache.spark.sql.errors.{QueryCompilationErrors, 
QueryExecutionErrors}
-import org.apache.spark.sql.types.{StringType, StructField, StructType}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{NullType, StringType, StructField, 
StructType}
 
 
 /**
@@ -101,6 +105,97 @@ case class DescribeFunctionCommand(
     toAttributes(schema)
   }
 
+  private def append(buffer: ArrayBuffer[(String, String)], key: String, 
value: String): Unit = {
+    buffer += (key -> value)
+  }
+
+  /**
+   * Pad all input strings to the same length using the max string length 
among all inputs.
+   */
+  private def tabulate(inputs: Seq[String]): Seq[String] = {
+    val maxLen = inputs.map(_.length).max
+    inputs.map { input => input.padTo(maxLen, " ").mkString }
+  }
+
+  private def formatParameters(params: StructType): Seq[String] = {
+    val names = tabulate(params.map(_.name))
+    val dataTypes = tabulate(params.map(_.dataType.sql))
+    // Only show parameter comments in extended mode.
+    val comments = params.map { p =>
+      if (isExtended) p.getComment().map(c => s" '$c'").getOrElse("") else ""
+    }
+    val defaults = params.map { p =>
+      if (isExtended) p.getDefault().map(d => s" DEFAULT $d").getOrElse("") 
else ""
+    }
+    names zip dataTypes zip defaults zip comments map {
+      case (((name, dataType), default), comment) => s"$name 
$dataType$default$comment"
+    }
+  }
+
+  private def describeSQLFunction(
+      info: ExpressionInfo,
+      qualifiedName: FunctionIdentifier,
+      parser: ParserInterface): Seq[Row] = {
+    val buffer = new ArrayBuffer[(String, String)]
+    val f = SQLFunction.fromExpressionInfo(info, parser)
+    // Match the legacy DESCRIBE FUNCTION path's qualification depth so
+    // `Function:` always renders the catalog-qualified 3-part name (when
+    // applicable), regardless of whether the function is a SQL UDF.
+    append(buffer, "Function:", qualifiedName.unquotedString)
+    append(buffer, "Type:", if (f.isTableFunc) SQLFunction.TABLE else 
SQLFunction.SCALAR)
+    // Function input
+    val input = f.inputParam
+    if (input.nonEmpty) {
+      val params = formatParameters(input.get)
+      assert(params.nonEmpty)
+      append(buffer, "Input:", params.head)
+      params.tail.foreach(s => append(buffer, "", s))
+    } else {
+      append(buffer, "Input:", "()")
+    }
+    // Function returns
+    if (f.isTableFunc) {
+      val returnParams = formatParameters(f.getTableFuncReturnCols)
+      assert(returnParams.nonEmpty)
+      append(buffer, "Returns:", returnParams.head)
+      returnParams.tail.foreach(s => append(buffer, "", s))
+    } else {
+      f.getScalarFuncReturnType match {
+        case _: NullType =>
+        case other => append(buffer, "Returns:", other.sql)
+      }
+    }
+    if (isExtended) {
+      f.comment.foreach(c => append(buffer, "Comment:", c))
+      f.collation.foreach(c => append(buffer, "Collation:", c))
+      f.deterministic.foreach(d => append(buffer, "Deterministic:", 
d.toString))
+      f.containsSQL.foreach { c =>
+        val dataAccess = if (c) "CONTAINS SQL" else "READS SQL DATA"
+        append(buffer, "Data Access:", dataAccess)
+      }
+      val configs = f.getSQLConfigs
+      if (configs.nonEmpty) {
+        val sorted = configs.toSeq.sortBy(_._1).map { case (key, value) => 
s"$key=$value" }
+        append(buffer, "Configs:", sorted.head)
+        sorted.tail.foreach(s => append(buffer, "", s))
+      }
+      f.owner.foreach(o => append(buffer, "Owner:", o))
+      append(buffer, "Create Time:", new 
java.util.Date(f.createTimeMs).toString)
+      // Put the function body at the end of the description.
+      append(buffer, "Body:", f.exprText.orElse(f.queryText).get)
+      // Show the frozen SQL PATH if one was persisted at function creation 
time.
+      if (SQLConf.get.pathEnabled) {
+        f.functionStoredResolutionPath
+          .flatMap(SqlPathFormat.toDescribeJson)
+          .flatMap(SqlPathFormat.formatForDisplay)
+          .foreach(p => append(buffer, "SQL Path:", p))
+      }
+    }
+    val keys = tabulate(buffer.map(_._1).toSeq)
+    val values = buffer.map(_._2)
+    keys.zip(values).map { case (key, value) => Row(s"$key $value") }
+  }
+
   override def run(sparkSession: SparkSession): Seq[Row] = {
     val identifier = if (info.getDb != null) {
       sparkSession.sessionState.catalog.qualifyIdentifier(
@@ -108,31 +203,23 @@ case class DescribeFunctionCommand(
     } else {
       FunctionIdentifier(info.getName)
     }
-    val name = identifier.unquotedString
-    val result = if (info.getClassName != null) {
-      Row(s"Function: $name") ::
-        Row(s"Class: ${info.getClassName}") ::
-        Row(s"Usage: ${info.getUsage}") :: Nil
+    if (SQLFunction.isSQLFunction(info.getClassName)) {
+      describeSQLFunction(info, identifier, 
sparkSession.sessionState.sqlParser)
     } else {
-      Row(s"Function: $name") :: Row(s"Usage: ${info.getUsage}") :: Nil
-    }
-
-    val sqlPathRows =
-      if (isExtended &&
-        sparkSession.sessionState.conf.pathEnabled &&
-        SQLFunction.isSQLFunction(info.getClassName)) {
-        DescribeFunctionCommandUtils
-          .storedResolutionPathString(sparkSession, identifier, info)
-          .map(s => Seq(Row(s"SQL Path: $s")))
-          .getOrElse(Nil)
+      val name = identifier.unquotedString
+      val result = if (info.getClassName != null) {
+        Row(s"Function: $name") ::
+          Row(s"Class: ${info.getClassName}") ::
+          Row(s"Usage: ${info.getUsage}") :: Nil
       } else {
-        Nil
+        Row(s"Function: $name") :: Row(s"Usage: ${info.getUsage}") :: Nil
       }
 
-    if (isExtended) {
-      (result ++ sqlPathRows) :+ Row(s"Extended Usage:${info.getExtended}")
-    } else {
-      result
+      if (isExtended) {
+        result :+ Row(s"Extended Usage:${info.getExtended}")
+      } else {
+        result
+      }
     }
   }
 }
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLFunctionSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLFunctionSuite.scala
index 9a3af9e1b432..4362064eb861 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLFunctionSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLFunctionSuite.scala
@@ -17,6 +17,9 @@
 
 package org.apache.spark.sql.execution
 
+import java.text.SimpleDateFormat
+import java.util.Locale
+
 import org.apache.spark.sql.{AnalysisException, Row}
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.test.SharedSparkSession
@@ -113,6 +116,127 @@ class SQLFunctionSuite extends SharedSparkSession {
     }
   }
 
+  test("describe SQL scalar functions") {
+    withUserDefinedFunction("foo" -> true, "bar" -> true, "area" -> false) {
+      // Temporary function
+      sql(
+        """
+          |CREATE TEMPORARY FUNCTION foo() RETURNS int
+          |COMMENT 'function foo' RETURN 1
+          |""".stripMargin)
+      checkKeywordsExist(sql("describe function foo"),
+        "Function:", "foo",
+        "Type:", "SCALAR",
+        "Input:", "()",
+        "Returns:", "INT")
+      checkKeywordsExist(sql("describe function extended foo"),
+        "Deterministic: true",
+        "Data Access:", "CONTAINS SQL",
+        "Comment:", "function foo",
+        "Create Time:",
+        "Body:", "1")
+      sql(
+        """
+          |CREATE TEMPORARY FUNCTION bar(x int default 8,
+          |y int default substr('8hello', 1, 1) comment 'var_y')
+          |RETURNS int COMMENT 'function bar' RETURN x + y
+          |""".stripMargin)
+      checkKeywordsExist(sql("describe function bar"),
+        "Function:", "bar",
+        "Input:", "x INT", "y INT",
+        "Returns:", "INT")
+      checkKeywordsExist(sql("describe function extended bar"),
+        "Input:", "x INT DEFAULT 8", "y INT DEFAULT substr('8hello', 1, 1) 
'var_y'",
+        "Comment:", "function bar",
+        "Deterministic: true",
+        "Data Access:", "CONTAINS SQL",
+        "Body:", "x + y")
+      // Permanent function
+      val beforeMs = System.currentTimeMillis()
+      sql(
+        """
+          |CREATE FUNCTION area(width double comment 'width', height double 
comment 'height')
+          |RETURNS double
+          |COMMENT 'compute area'
+          |DETERMINISTIC
+          |RETURN width * height
+          |""".stripMargin)
+      val afterMs = System.currentTimeMillis()
+      checkKeywordsExist(sql("describe function area"),
+        "Function:", "default.area",
+        "Type:", "SCALAR",
+        "Input:", "width  DOUBLE", "height DOUBLE",
+        "Returns:", "DOUBLE")
+      val extendedRows = sql("describe function extended area").collect()
+      checkKeywordsExist(sql("describe function extended area"),
+        "Input:", "width  DOUBLE 'width'", "height DOUBLE 'height'",
+        "Comment:", "compute area",
+        "Deterministic: true",
+        "Data Access:", "CONTAINS SQL",
+        "Create Time:",
+        "Body:", "width * height")
+      // Verify the rendered Create Time falls within a small window around the
+      // CREATE FUNCTION call, i.e. the timestamp set at CREATE time was 
preserved
+      // (and not silently overwritten by a later cache-build / metadata-load).
+      val createTimeRow = extendedRows.map(_.getString(0))
+        .find(_.startsWith("Create Time:"))
+        .getOrElse(fail("DESCRIBE FUNCTION EXTENDED is missing the Create Time 
row"))
+      val tsStr = createTimeRow.split("Create Time:", 2)(1).trim
+      // Date.toString() format -- explicit Locale.ENGLISH avoids parser drift 
on
+      // build hosts whose default locale is not English.
+      val sdf = new SimpleDateFormat("EEE MMM dd HH:mm:ss zzz yyyy", 
Locale.ENGLISH)
+      val parsedMs = sdf.parse(tsStr).getTime
+      // Date.toString() truncates to seconds; use a 2-second slop on each 
side.
+      val slopMs = 2000L
+      assert(parsedMs >= beforeMs - slopMs,
+        s"Create Time '$tsStr' is before CREATE FUNCTION (beforeMs=$beforeMs)")
+      assert(parsedMs <= afterMs + slopMs,
+        s"Create Time '$tsStr' is after DESCRIBE FUNCTION (afterMs=$afterMs)")
+    }
+  }
+
+  test("describe SQL table functions") {
+    withUserDefinedFunction("foo" -> false) {
+      sql(
+        """
+          |CREATE FUNCTION foo(x INT) RETURNS TABLE (a INT, b STRING)
+          |COMMENT 'table function foo' RETURN SELECT x, x
+          |""".stripMargin)
+      checkKeywordsExist(sql("describe function foo"),
+        "Function:", "foo",
+        "Type:", "TABLE",
+        "Input:", "x INT",
+        "Returns:", "a INT", "b STRING")
+      checkKeywordsExist(sql("describe function extended foo"),
+        "Comment:", "table function foo",
+        "Deterministic: true",
+        "Data Access:", "CONTAINS SQL",
+        "Create Time:",
+        "Body:", "SELECT x, x")
+    }
+  }
+
+  test("describe SQL functions with derived routine characteristics") {
+    withUserDefinedFunction("foo" -> false, "bar" -> false, "baz" -> false) {
+      withTable("tbl_for_describe") {
+        sql("CREATE TABLE tbl_for_describe AS SELECT 1 AS x")
+        sql("CREATE FUNCTION foo() RETURNS TABLE(x INT) RETURN SELECT * FROM 
tbl_for_describe")
+        sql("CREATE FUNCTION bar() RETURNS DOUBLE RETURN SELECT SUM(x) + 
rand() FROM foo()")
+        sql("CREATE FUNCTION baz() RETURNS INT NOT DETERMINISTIC READS SQL 
DATA RETURN 1")
+        checkKeywordsExist(sql("DESCRIBE FUNCTION EXTENDED foo"),
+          "Deterministic: true",
+          "Data Access:", "READS SQL DATA")
+        checkKeywordsExist(sql("DESCRIBE FUNCTION EXTENDED bar"),
+          "Deterministic: false",
+          "Data Access:", "READS SQL DATA")
+        // Do not overwrite user-specified routine characteristics.
+        checkKeywordsExist(sql("DESCRIBE FUNCTION EXTENDED baz"),
+          "Deterministic: false",
+          "Data Access:", "READS SQL DATA")
+      }
+    }
+  }
+
   test("SPARK-56639: SQL function uses frozen SQL path") {
     withSQLConf(SQLConf.PATH_ENABLED.key -> "true") {
       withDatabase("path_func_db_a", "path_func_db_b") {
@@ -135,6 +259,15 @@ class SQLFunctionSuite extends SharedSparkSession {
 
               checkAnswer(sql("SELECT MAX(id) FROM frozen_t"), Row(20))
               checkAnswer(sql("SELECT default.frozen_fn()"), Row(10))
+              // DESCRIBE FUNCTION EXTENDED renders the frozen creator path,
+              // not the invoker's current PATH. SqlPathFormat.formatForDisplay
+              // back-ticks identifiers only when needed, so plain ASCII
+              // identifiers appear unquoted.
+              checkKeywordsExist(sql("DESCRIBE FUNCTION EXTENDED 
default.frozen_fn"),
+                "SQL Path:",
+                "spark_catalog.path_func_db_a, system.builtin")
+              checkKeywordsNotExist(sql("DESCRIBE FUNCTION EXTENDED 
default.frozen_fn"),
+                "path_func_db_b")
             } finally {
               sql("SET PATH = DEFAULT_PATH")
             }


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to