You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by gu...@apache.org on 2020/04/21 01:56:31 UTC

[spark] branch master updated: [SPARK-31429][SQL][DOC] Automatically generates a SQL document for built-in functions

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

gurwls223 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 e42dbe7  [SPARK-31429][SQL][DOC] Automatically generates a SQL document for built-in functions
e42dbe7 is described below

commit e42dbe7cd41c3689910165458a92b75c02e70a03
Author: Takeshi Yamamuro <ya...@apache.org>
AuthorDate: Tue Apr 21 10:55:13 2020 +0900

    [SPARK-31429][SQL][DOC] Automatically generates a SQL document for built-in functions
    
    ### What changes were proposed in this pull request?
    
    This PR intends to add a Python script to generates a SQL document for built-in functions and the document in SQL references.
    
    ### Why are the changes needed?
    
    To make SQL references complete.
    
    ### Does this PR introduce any user-facing change?
    
    Yes;
    
    ![a](https://user-images.githubusercontent.com/692303/79406712-c39e1b80-7fd2-11ea-8b85-9f9cbb6efed3.png)
    ![b](https://user-images.githubusercontent.com/692303/79320526-eb46a280-7f44-11ea-8639-90b1fb2b8848.png)
    ![c](https://user-images.githubusercontent.com/692303/79320707-3365c500-7f45-11ea-9984-69ffe800fb87.png)
    
    ### How was this patch tested?
    
    Manually checked and added tests.
    
    Closes #28224 from maropu/SPARK-31429.
    
    Lead-authored-by: Takeshi Yamamuro <ya...@apache.org>
    Co-authored-by: HyukjinKwon <gu...@apache.org>
    Signed-off-by: HyukjinKwon <gu...@apache.org>
---
 docs/.gitignore                                    |   2 +-
 docs/_data/menu-sql.yaml                           |   2 +
 docs/configuration.md                              |   4 +-
 docs/sql-ref-functions-builtin.md                  |  77 +++++++
 docs/sql-ref-functions.md                          |  12 ++
 .../expressions/ExpressionDescription.java         |  14 +-
 .../sql/catalyst/expressions/ExpressionInfo.java   |  27 ++-
 .../sql/catalyst/analysis/FunctionRegistry.scala   |   3 +-
 .../aggregate/ApproximatePercentile.scala          |   1 +
 .../catalyst/expressions/aggregate/Average.scala   |   1 +
 .../expressions/aggregate/CentralMomentAgg.scala   |   6 +
 .../sql/catalyst/expressions/aggregate/Corr.scala  |   1 +
 .../sql/catalyst/expressions/aggregate/Count.scala |   1 +
 .../catalyst/expressions/aggregate/CountIf.scala   |   1 +
 .../expressions/aggregate/CountMinSketchAgg.scala  |   1 +
 .../expressions/aggregate/Covariance.scala         |   2 +
 .../sql/catalyst/expressions/aggregate/First.scala |   1 +
 .../aggregate/HyperLogLogPlusPlus.scala            |   1 +
 .../sql/catalyst/expressions/aggregate/Last.scala  |   1 +
 .../sql/catalyst/expressions/aggregate/Max.scala   |   1 +
 .../expressions/aggregate/MaxByAndMinBy.scala      |   2 +
 .../sql/catalyst/expressions/aggregate/Min.scala   |   1 +
 .../expressions/aggregate/Percentile.scala         |   1 +
 .../sql/catalyst/expressions/aggregate/Sum.scala   |   1 +
 .../expressions/aggregate/UnevaluableAggs.scala    |   2 +
 .../expressions/aggregate/bitwiseAggregates.scala  |   2 +
 .../catalyst/expressions/aggregate/collect.scala   |   2 +
 .../expressions/collectionOperations.scala         |  59 ++++--
 .../catalyst/expressions/complexTypeCreator.scala  |   1 +
 .../catalyst/expressions/datetimeExpressions.scala |  32 +++
 .../sql/catalyst/expressions/jsonExpressions.scala |  17 +-
 .../catalyst/expressions/windowExpressions.scala   |  24 ++-
 .../spark/sql/SparkSessionExtensionSuite.scala     |   3 +
 .../test/scala/org/apache/spark/sql/UDFSuite.scala |  19 ++
 sql/create-docs.sh                                 |   3 +
 sql/gen-sql-api-docs.py                            |   5 +-
 sql/gen-sql-config-docs.py                         |   9 +-
 sql/gen-sql-functions-docs.py                      | 228 +++++++++++++++++++++
 38 files changed, 528 insertions(+), 42 deletions(-)

diff --git a/docs/.gitignore b/docs/.gitignore
index 2260493..9df83f3 100644
--- a/docs/.gitignore
+++ b/docs/.gitignore
@@ -1 +1 @@
-sql-configs.html
+generated-*.html
diff --git a/docs/_data/menu-sql.yaml b/docs/_data/menu-sql.yaml
index 5042c2588..a16e114 100644
--- a/docs/_data/menu-sql.yaml
+++ b/docs/_data/menu-sql.yaml
@@ -246,6 +246,8 @@
     - text: Functions
       url: sql-ref-functions.html
       subitems:
+      - text: Built-in Functions
+        url: sql-ref-functions-builtin.html
       - text: Scalar UDFs (User-Defined Functions)
         url: sql-ref-functions-udf-scalar.html
       - text: UDAFs (User-Defined Aggregate Functions)
diff --git a/docs/configuration.md b/docs/configuration.md
index 676ecf5..e322247 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -2623,10 +2623,10 @@ Spark subsystems.
 
 
 {% for static_file in site.static_files %}
-    {% if static_file.name == 'sql-configs.html' %}
+    {% if static_file.name == 'generated-sql-configuration-table.html' %}
 ### Spark SQL
 
-        {% include_relative sql-configs.html %}
+{% include_relative generated-sql-configuration-table.html %}
         {% break %}
     {% endif %}
 {% endfor %}
diff --git a/docs/sql-ref-functions-builtin.md b/docs/sql-ref-functions-builtin.md
new file mode 100644
index 0000000..1bca68e
--- /dev/null
+++ b/docs/sql-ref-functions-builtin.md
@@ -0,0 +1,77 @@
+---
+layout: global
+title: Built-in Functions
+displayTitle: Built-in Functions
+license: |
+  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.
+---
+
+{% for static_file in site.static_files %}
+    {% if static_file.name == 'generated-agg-funcs-table.html' %}
+### Aggregate Functions
+{% include_relative generated-agg-funcs-table.html %}
+#### Examples
+{% include_relative generated-agg-funcs-examples.html %}
+        {% break %}
+    {% endif %}
+{% endfor %}
+
+{% for static_file in site.static_files %}
+    {% if static_file.name == 'generated-window-funcs-table.html' %}
+### Window Functions
+{% include_relative generated-window-funcs-table.html %}
+        {% break %}
+    {% endif %}
+{% endfor %}
+
+{% for static_file in site.static_files %}
+    {% if static_file.name == 'generated-array-funcs-table.html' %}
+### Array Functions
+{% include_relative generated-array-funcs-table.html %}
+#### Examples
+{% include_relative generated-array-funcs-examples.html %}
+        {% break %}
+    {% endif %}
+{% endfor %}
+
+{% for static_file in site.static_files %}
+    {% if static_file.name == 'generated-map-funcs-table.html' %}
+### Map Functions
+{% include_relative generated-map-funcs-table.html %}
+#### Examples
+{% include_relative generated-map-funcs-examples.html %}
+        {% break %}
+    {% endif %}
+{% endfor %}
+
+{% for static_file in site.static_files %}
+    {% if static_file.name == 'generated-datetime-funcs-table.html' %}
+### Date and Timestamp Functions
+{% include_relative generated-datetime-funcs-table.html %}
+#### Examples
+{% include_relative generated-datetime-funcs-examples.html %}
+        {% break %}
+    {% endif %}
+{% endfor %}
+
+{% for static_file in site.static_files %}
+    {% if static_file.name == 'generated-json-funcs-table.html' %}
+### JSON Functions
+{% include_relative generated-json-funcs-table.html %}
+#### Examples
+{% include_relative generated-agg-funcs-examples.html %}
+        {% break %}
+    {% endif %}
+{% endfor %}
+
diff --git a/docs/sql-ref-functions.md b/docs/sql-ref-functions.md
index e8a0353..6368fb7 100644
--- a/docs/sql-ref-functions.md
+++ b/docs/sql-ref-functions.md
@@ -22,6 +22,18 @@ license: |
 Spark SQL provides two function features to meet a wide range of user needs: built-in functions and user-defined functions (UDFs).
 Built-in functions are commonly used routines that Spark SQL predefines and a complete list of the functions can be found in the [Built-in Functions](api/sql/) API document. UDFs allow users to define their own functions when the system’s built-in functions are not enough to perform the desired task.
 
+### Built-in Functions
+
+Spark SQL has some categories of frequently-used built-in functions for aggregtion, arrays/maps, date/timestamp, and JSON data.
+This subsection presents the usages and descriptions of these functions.
+
+ * [Aggregate Functions](sql-ref-functions-builtin.html#aggregate-functions)
+ * [Window Functions](sql-ref-functions-builtin.html#window-functions)
+ * [Array Functions](sql-ref-functions-builtin.html#array-functions)
+ * [Map Functions](sql-ref-functions-builtin.html#map-functions)
+ * [Date and Timestamp Functions](sql-ref-functions-builtin.html#date-and-timestamp-functions)
+ * [JSON Functions](sql-ref-functions-builtin.html#json-functions)
+
 ### UDFs (User-Defined Functions)
 
 User-Defined Functions (UDFs) are a feature of Spark SQL that allows users to define their own functions when the system's built-in functions are not enough to perform the desired task. To use UDFs in Spark SQL, users must first define the function, then register the function with Spark, and finally call the registered function. The User-Defined Functions can act on a single row or act on multiple rows at once. Spark SQL also supports integration of existing Hive implementations of UDFs, [...]
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionDescription.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionDescription.java
index acdf6af..0f648ab 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionDescription.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionDescription.java
@@ -31,7 +31,7 @@ import java.lang.annotation.RetentionPolicy;
  * `usage()` will be used for the function usage in brief way.
  *
  * These below are concatenated and used for the function usage in verbose way, suppose arguments,
- * examples, note, since and deprecated will be provided.
+ * examples, note, group, since and deprecated will be provided.
  *
  * `arguments()` describes arguments for the expression.
  *
@@ -39,13 +39,16 @@ import java.lang.annotation.RetentionPolicy;
  *
  * `note()` contains some notes for the expression optionally.
  *
+ * `group()` describes the category that the expression belongs to. The valid value is
+ * "agg_funcs", "array_funcs", "datetime_funcs", "json_funcs", "map_funcs" and "window_funcs".
+ *
  * `since()` contains version information for the expression. Version is specified by,
  * for example, "2.2.0".
  *
  * `deprecated()` contains deprecation information for the expression optionally, for example,
  * "Deprecated since 2.2.0. Use something else instead".
  *
- * The format, in particular for `arguments()`, `examples()`,`note()`, `since()` and
+ * The format, in particular for `arguments()`, `examples()`,`note()`, `group()`, `since()` and
  * `deprecated()`, should strictly be as follows.
  *
  * <pre>
@@ -68,6 +71,7 @@ import java.lang.annotation.RetentionPolicy;
  *   note = """
  *     ...
  *   """,
+ *   group = "agg_funcs",
  *   since = "3.0.0",
  *   deprecated = """
  *     ...
@@ -78,8 +82,9 @@ import java.lang.annotation.RetentionPolicy;
  *  We can refer the function name by `_FUNC_`, in `usage()`, `arguments()` and `examples()` as
  *  it is registered in `FunctionRegistry`.
  *
- *  Note that, if `extended()` is defined, `arguments()`, `examples()`, `note()`, `since()` and
- *  `deprecated()` should be not defined together. `extended()` exists for backward compatibility.
+ *  Note that, if `extended()` is defined, `arguments()`, `examples()`, `note()`, `group()`,
+ *  `since()` and `deprecated()` should be not defined together. `extended()` exists
+ *  for backward compatibility.
  *
  *  Note this contents are used in the SparkSQL documentation for built-in functions. The contents
  *  here are considered as a Markdown text and then rendered.
@@ -98,6 +103,7 @@ public @interface ExpressionDescription {
     String arguments() default "";
     String examples() default "";
     String note() default "";
+    String group() default "";
     String since() default "";
     String deprecated() default "";
 }
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java
index 8ee90ed..a500822 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java
@@ -19,6 +19,10 @@ package org.apache.spark.sql.catalyst.expressions;
 
 import com.google.common.annotations.VisibleForTesting;
 
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+
 /**
  * Expression information, will be used to describe a expression.
  */
@@ -31,9 +35,14 @@ public class ExpressionInfo {
     private String arguments;
     private String examples;
     private String note;
+    private String group;
     private String since;
     private String deprecated;
 
+    private static final Set<String> validGroups =
+        new HashSet<>(Arrays.asList("agg_funcs", "array_funcs", "datetime_funcs",
+            "json_funcs", "map_funcs", "window_funcs"));
+
     public String getClassName() {
         return className;
     }
@@ -75,6 +84,10 @@ public class ExpressionInfo {
         return deprecated;
     }
 
+    public String getGroup() {
+        return group;
+    }
+
     public String getDb() {
         return db;
     }
@@ -87,6 +100,7 @@ public class ExpressionInfo {
             String arguments,
             String examples,
             String note,
+            String group,
             String since,
             String deprecated) {
         assert name != null;
@@ -94,6 +108,7 @@ public class ExpressionInfo {
         assert examples != null;
         assert examples.isEmpty() || examples.contains("    Examples:");
         assert note != null;
+        assert group != null;
         assert since != null;
         assert deprecated != null;
 
@@ -104,6 +119,7 @@ public class ExpressionInfo {
         this.arguments = arguments;
         this.examples = examples;
         this.note = note;
+        this.group = group;
         this.since = since;
         this.deprecated = deprecated;
 
@@ -120,6 +136,11 @@ public class ExpressionInfo {
             }
             this.extended += "\n    Note:\n      " + note.trim() + "\n";
         }
+        if (!group.isEmpty() && !validGroups.contains(group)) {
+            throw new IllegalArgumentException("'group' is malformed in the expression [" +
+                this.name + "]. It should be a value in " + validGroups + "; however, " +
+                "got [" + group + "].");
+        }
         if (!since.isEmpty()) {
             if (Integer.parseInt(since.split("\\.")[0]) < 0) {
                 throw new IllegalArgumentException("'since' is malformed in the expression [" +
@@ -140,11 +161,11 @@ public class ExpressionInfo {
     }
 
     public ExpressionInfo(String className, String name) {
-        this(className, null, name, null, "", "", "", "", "");
+        this(className, null, name, null, "", "", "", "", "", "");
     }
 
     public ExpressionInfo(String className, String db, String name) {
-        this(className, db, name, null, "", "", "", "", "");
+        this(className, db, name, null, "", "", "", "", "", "");
     }
 
     /**
@@ -155,7 +176,7 @@ public class ExpressionInfo {
     public ExpressionInfo(String className, String db, String name, String usage, String extended) {
         // `arguments` and `examples` are concatenated for the extended description. So, here
         // simply pass the `extended` as `arguments` and an empty string for `examples`.
-        this(className, db, name, usage, extended, "", "", "", "");
+        this(className, db, name, usage, extended, "", "", "", "", "");
     }
 
     private String replaceFunctionName(String usage) {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
index 7f879c6..c0e7e78 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
@@ -655,7 +655,7 @@ object FunctionRegistry {
     val clazz = scala.reflect.classTag[Cast].runtimeClass
     val usage = "_FUNC_(expr) - Casts the value `expr` to the target data type `_FUNC_`."
     val expressionInfo =
-      new ExpressionInfo(clazz.getCanonicalName, null, name, usage, "", "", "", "", "")
+      new ExpressionInfo(clazz.getCanonicalName, null, name, usage, "", "", "", "", "", "")
     (name, (expressionInfo, builder))
   }
 
@@ -675,6 +675,7 @@ object FunctionRegistry {
           df.arguments(),
           df.examples(),
           df.note(),
+          df.group(),
           df.since(),
           df.deprecated())
       } else {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
index f806095..d06eeee 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
@@ -65,6 +65,7 @@ import org.apache.spark.sql.types._
       > SELECT _FUNC_(10.0, 0.5, 100);
        10.0
   """,
+  group = "agg_funcs",
   since = "2.1.0")
 case class ApproximatePercentile(
     child: Expression,
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala
index 17f906c..d3ce1f8 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala
@@ -32,6 +32,7 @@ import org.apache.spark.sql.types._
       > SELECT _FUNC_(col) FROM VALUES (1), (2), (NULL) AS tab(col);
        1.5
   """,
+  group = "agg_funcs",
   since = "1.0.0")
 case class Average(child: Expression) extends DeclarativeAggregate with ImplicitCastInputTypes {
 
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala
index bf40280..53759ca 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala
@@ -142,6 +142,7 @@ abstract class CentralMomentAgg(child: Expression)
       > SELECT _FUNC_(col) FROM VALUES (1), (2), (3) AS tab(col);
        0.816496580927726
   """,
+  group = "agg_funcs",
   since = "1.6.0")
 // scalastyle:on line.size.limit
 case class StddevPop(child: Expression) extends CentralMomentAgg(child) {
@@ -164,6 +165,7 @@ case class StddevPop(child: Expression) extends CentralMomentAgg(child) {
       > SELECT _FUNC_(col) FROM VALUES (1), (2), (3) AS tab(col);
        1.0
   """,
+  group = "agg_funcs",
   since = "1.6.0")
 // scalastyle:on line.size.limit
 case class StddevSamp(child: Expression) extends CentralMomentAgg(child) {
@@ -187,6 +189,7 @@ case class StddevSamp(child: Expression) extends CentralMomentAgg(child) {
       > SELECT _FUNC_(col) FROM VALUES (1), (2), (3) AS tab(col);
        0.6666666666666666
   """,
+  group = "agg_funcs",
   since = "1.6.0")
 case class VariancePop(child: Expression) extends CentralMomentAgg(child) {
 
@@ -207,6 +210,7 @@ case class VariancePop(child: Expression) extends CentralMomentAgg(child) {
       > SELECT _FUNC_(col) FROM VALUES (1), (2), (3) AS tab(col);
        1.0
   """,
+  group = "agg_funcs",
   since = "1.6.0")
 case class VarianceSamp(child: Expression) extends CentralMomentAgg(child) {
 
@@ -229,6 +233,7 @@ case class VarianceSamp(child: Expression) extends CentralMomentAgg(child) {
       > SELECT _FUNC_(col) FROM VALUES (-1000), (-100), (10), (20) AS tab(col);
        -1.1135657469022011
   """,
+  group = "agg_funcs",
   since = "1.6.0")
 case class Skewness(child: Expression) extends CentralMomentAgg(child) {
 
@@ -251,6 +256,7 @@ case class Skewness(child: Expression) extends CentralMomentAgg(child) {
       > SELECT _FUNC_(col) FROM VALUES (1), (10), (100), (10), (1) as tab(col);
        0.19432323191699075
   """,
+  group = "agg_funcs",
   since = "1.6.0")
 case class Kurtosis(child: Expression) extends CentralMomentAgg(child) {
 
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Corr.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Corr.scala
index 91446e0..9ef05bb 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Corr.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Corr.scala
@@ -99,6 +99,7 @@ abstract class PearsonCorrelation(x: Expression, y: Expression)
       > SELECT _FUNC_(c1, c2) FROM VALUES (3, 2), (3, 3), (6, 4) as tab(c1, c2);
        0.8660254037844387
   """,
+  group = "agg_funcs",
   since = "1.6.0")
 // scalastyle:on line.size.limit
 case class Corr(x: Expression, y: Expression)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Count.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Count.scala
index 2a8edac..e043c81 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Count.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Count.scala
@@ -39,6 +39,7 @@ import org.apache.spark.sql.types._
       > SELECT _FUNC_(DISTINCT col) FROM VALUES (NULL), (5), (5), (10) AS tab(col);
        2
   """,
+  group = "agg_funcs",
   since = "1.0.0")
 // scalastyle:on line.size.limit
 case class Count(children: Seq[Expression]) extends DeclarativeAggregate {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CountIf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CountIf.scala
index d31355c..5bb95ea 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CountIf.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CountIf.scala
@@ -32,6 +32,7 @@ import org.apache.spark.sql.types.{AbstractDataType, BooleanType, DataType, Long
       > SELECT _FUNC_(col IS NULL) FROM VALUES (NULL), (0), (1), (2), (3) AS tab(col);
        1
   """,
+  group = "agg_funcs",
   since = "3.0.0")
 case class CountIf(predicate: Expression) extends UnevaluableAggregate with ImplicitCastInputTypes {
   override def prettyName: String = "count_if"
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CountMinSketchAgg.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CountMinSketchAgg.scala
index 4bd13cf..787b218 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CountMinSketchAgg.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CountMinSketchAgg.scala
@@ -44,6 +44,7 @@ import org.apache.spark.util.sketch.CountMinSketch
       `CountMinSketch` before usage. Count-min sketch is a probabilistic data structure used for
       cardinality estimation using sub-linear space.
   """,
+  group = "agg_funcs",
   since = "2.2.0")
 case class CountMinSketchAgg(
     child: Expression,
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala
index ac99fa8..f03c2f2 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala
@@ -86,6 +86,7 @@ abstract class Covariance(x: Expression, y: Expression)
       > SELECT _FUNC_(c1, c2) FROM VALUES (1,1), (2,2), (3,3) AS tab(c1, c2);
        0.6666666666666666
   """,
+  group = "agg_funcs",
   since = "2.0.0")
 case class CovPopulation(left: Expression, right: Expression) extends Covariance(left, right) {
   override val evaluateExpression: Expression = {
@@ -102,6 +103,7 @@ case class CovPopulation(left: Expression, right: Expression) extends Covariance
       > SELECT _FUNC_(c1, c2) FROM VALUES (1,1), (2,2), (3,3) AS tab(c1, c2);
        1.0
   """,
+  group = "agg_funcs",
   since = "2.0.0")
 case class CovSample(left: Expression, right: Expression) extends Covariance(left, right) {
   override val evaluateExpression: Expression = {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/First.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/First.scala
index 2c0060c..df806ed 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/First.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/First.scala
@@ -47,6 +47,7 @@ import org.apache.spark.sql.types._
     The function is non-deterministic because its results depends on the order of the rows
     which may be non-deterministic after a shuffle.
   """,
+  group = "agg_funcs",
   since = "2.0.0")
 case class First(child: Expression, ignoreNullsExpr: Expression)
   extends DeclarativeAggregate with ExpectsInputTypes {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlus.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlus.scala
index e3c628e..aed3690 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlus.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlus.scala
@@ -53,6 +53,7 @@ import org.apache.spark.sql.types._
       > SELECT _FUNC_(col1) FROM VALUES (1), (1), (2), (2), (3) tab(col1);
        3
   """,
+  group = "agg_funcs",
   since = "1.6.0")
 case class HyperLogLogPlusPlus(
     child: Expression,
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala
index 6793ac7..e55bced 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala
@@ -47,6 +47,7 @@ import org.apache.spark.sql.types._
     The function is non-deterministic because its results depends on the order of the rows
     which may be non-deterministic after a shuffle.
   """,
+  group = "agg_funcs",
   since = "2.0.0")
 case class Last(child: Expression, ignoreNullsExpr: Expression)
   extends DeclarativeAggregate with ExpectsInputTypes {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Max.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Max.scala
index 7520db1..9bba660 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Max.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Max.scala
@@ -30,6 +30,7 @@ import org.apache.spark.sql.types._
       > SELECT _FUNC_(col) FROM VALUES (10), (50), (20) AS tab(col);
        50
   """,
+  group = "agg_funcs",
   since = "1.0.0")
 case class Max(child: Expression) extends DeclarativeAggregate {
 
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/MaxByAndMinBy.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/MaxByAndMinBy.scala
index b69b341..2e20224 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/MaxByAndMinBy.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/MaxByAndMinBy.scala
@@ -98,6 +98,7 @@ abstract class MaxMinBy extends DeclarativeAggregate {
       > SELECT _FUNC_(x, y) FROM VALUES (('a', 10)), (('b', 50)), (('c', 20)) AS tab(x, y);
        b
   """,
+  group = "agg_funcs",
   since = "3.0.0")
 case class MaxBy(valueExpr: Expression, orderingExpr: Expression) extends MaxMinBy {
   override protected def funcName: String = "max_by"
@@ -116,6 +117,7 @@ case class MaxBy(valueExpr: Expression, orderingExpr: Expression) extends MaxMin
       > SELECT _FUNC_(x, y) FROM VALUES (('a', 10)), (('b', 50)), (('c', 20)) AS tab(x, y);
        a
   """,
+  group = "agg_funcs",
   since = "3.0.0")
 case class MinBy(valueExpr: Expression, orderingExpr: Expression) extends MaxMinBy {
   override protected def funcName: String = "min_by"
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Min.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Min.scala
index 106eb96..1d861aa 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Min.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Min.scala
@@ -30,6 +30,7 @@ import org.apache.spark.sql.types._
       > SELECT _FUNC_(col) FROM VALUES (10), (-1), (20) AS tab(col);
        -1
   """,
+  group = "agg_funcs",
   since = "1.0.0")
 case class Min(child: Expression) extends DeclarativeAggregate {
 
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Percentile.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Percentile.scala
index 0f1c0fb..0eba61c 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Percentile.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Percentile.scala
@@ -62,6 +62,7 @@ import org.apache.spark.util.collection.OpenHashMap
       > SELECT _FUNC_(col, array(0.25, 0.75)) FROM VALUES (0), (10) AS tab(col);
        [2.5,7.5]
   """,
+  group = "agg_funcs",
   since = "2.1.0")
 case class Percentile(
     child: Expression,
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala
index 8bfd889..d2daaac 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala
@@ -35,6 +35,7 @@ import org.apache.spark.sql.types._
       > SELECT _FUNC_(col) FROM VALUES (NULL), (NULL) AS tab(col);
        NULL
   """,
+  group = "agg_funcs",
   since = "1.0.0")
 case class Sum(child: Expression) extends DeclarativeAggregate with ImplicitCastInputTypes {
 
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/UnevaluableAggs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/UnevaluableAggs.scala
index a1cd4a7..cb77ded 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/UnevaluableAggs.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/UnevaluableAggs.scala
@@ -51,6 +51,7 @@ abstract class UnevaluableBooleanAggBase(arg: Expression)
       > SELECT _FUNC_(col) FROM VALUES (true), (false), (true) AS tab(col);
        false
   """,
+  group = "agg_funcs",
   since = "3.0.0")
 case class BoolAnd(arg: Expression) extends UnevaluableBooleanAggBase(arg) {
   override def nodeName: String = getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse("bool_and")
@@ -67,6 +68,7 @@ case class BoolAnd(arg: Expression) extends UnevaluableBooleanAggBase(arg) {
       > SELECT _FUNC_(col) FROM VALUES (false), (false), (NULL) AS tab(col);
        false
   """,
+  group = "agg_funcs",
   since = "3.0.0")
 case class BoolOr(arg: Expression) extends UnevaluableBooleanAggBase(arg) {
   override def nodeName: String = getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse("bool_or")
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/bitwiseAggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/bitwiseAggregates.scala
index b77c3bd..b4c1b2c 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/bitwiseAggregates.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/bitwiseAggregates.scala
@@ -77,6 +77,7 @@ case class BitAndAgg(child: Expression) extends BitAggregate {
       > SELECT _FUNC_(col) FROM VALUES (3), (5) AS tab(col);
        7
   """,
+  group = "agg_funcs",
   since = "3.0.0")
 case class BitOrAgg(child: Expression) extends BitAggregate {
 
@@ -94,6 +95,7 @@ case class BitOrAgg(child: Expression) extends BitAggregate {
       > SELECT _FUNC_(col) FROM VALUES (3), (5) AS tab(col);
        6
   """,
+  group = "agg_funcs",
   since = "3.0.0")
 case class BitXorAgg(child: Expression) extends BitAggregate {
 
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala
index 29f8998..5848aa3 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala
@@ -96,6 +96,7 @@ abstract class Collect[T <: Growable[Any] with Iterable[Any]] extends TypedImper
     The function is non-deterministic because the order of collected results depends
     on the order of the rows which may be non-deterministic after a shuffle.
   """,
+  group = "agg_funcs",
   since = "2.0.0")
 case class CollectList(
     child: Expression,
@@ -129,6 +130,7 @@ case class CollectList(
     The function is non-deterministic because the order of collected results depends
     on the order of the rows which may be non-deterministic after a shuffle.
   """,
+  group = "agg_funcs",
   since = "2.0.0")
 case class CollectSet(
     child: Expression,
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
index 8b61bc4..4fd68dc 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
@@ -138,7 +138,8 @@ object Size {
     Examples:
       > SELECT _FUNC_(map(1, 'a', 2, 'b'));
        [1,2]
-  """)
+  """,
+  group = "map_funcs")
 case class MapKeys(child: Expression)
   extends UnaryExpression with ExpectsInputTypes {
 
@@ -169,6 +170,7 @@ case class MapKeys(child: Expression)
       > SELECT _FUNC_(array(1, 2), array(2, 3), array(3, 4));
        [{"0":1,"1":2,"2":3},{"0":2,"1":3,"2":4}]
   """,
+  group = "array_funcs",
   since = "2.4.0")
 case class ArraysZip(children: Seq[Expression]) extends Expression with ExpectsInputTypes {
 
@@ -327,7 +329,8 @@ case class ArraysZip(children: Seq[Expression]) extends Expression with ExpectsI
     Examples:
       > SELECT _FUNC_(map(1, 'a', 2, 'b'));
        ["a","b"]
-  """)
+  """,
+  group = "map_funcs")
 case class MapValues(child: Expression)
   extends UnaryExpression with ExpectsInputTypes {
 
@@ -356,6 +359,7 @@ case class MapValues(child: Expression)
       > SELECT _FUNC_(map(1, 'a', 2, 'b'));
        [{"key":1,"value":"a"},{"key":2,"value":"b"}]
   """,
+  group = "map_funcs",
   since = "3.0.0")
 case class MapEntries(child: Expression) extends UnaryExpression with ExpectsInputTypes {
 
@@ -523,7 +527,9 @@ case class MapEntries(child: Expression) extends UnaryExpression with ExpectsInp
     Examples:
       > SELECT _FUNC_(map(1, 'a', 2, 'b'), map(3, 'c'));
        {1:"a",2:"b",3:"c"}
-  """, since = "2.4.0")
+  """,
+  group = "map_funcs",
+  since = "2.4.0")
 case class MapConcat(children: Seq[Expression]) extends ComplexTypeMergingExpression {
 
   override def checkInputDataTypes(): TypeCheckResult = {
@@ -641,6 +647,7 @@ case class MapConcat(children: Seq[Expression]) extends ComplexTypeMergingExpres
       > SELECT _FUNC_(array(struct(1, 'a'), struct(2, 'b')));
        {1:"a",2:"b"}
   """,
+  group = "map_funcs",
   since = "2.4.0")
 case class MapFromEntries(child: Expression) extends UnaryExpression {
 
@@ -862,7 +869,8 @@ object ArraySortLike {
     Examples:
       > SELECT _FUNC_(array('b', 'd', null, 'c', 'a'), true);
        [null,"a","b","c","d"]
-  """)
+  """,
+  group = "array_funcs")
 // scalastyle:on line.size.limit
 case class SortArray(base: Expression, ascendingOrder: Expression)
   extends BinaryExpression with ArraySortLike {
@@ -920,6 +928,7 @@ case class SortArray(base: Expression, ascendingOrder: Expression)
   note = """
     The function is non-deterministic.
   """,
+  group = "array_funcs",
   since = "2.4.0")
 case class Shuffle(child: Expression, randomSeed: Option[Long] = None)
   extends UnaryExpression with ExpectsInputTypes with Stateful with ExpressionWithRandomSeed {
@@ -1002,6 +1011,7 @@ case class Shuffle(child: Expression, randomSeed: Option[Long] = None)
       > SELECT _FUNC_(array(2, 1, 4, 3));
        [3,4,1,2]
   """,
+  group = "array_funcs",
   since = "1.5.0",
   note = """
     Reverse logic for arrays is available since 2.4.0.
@@ -1073,7 +1083,8 @@ case class Reverse(child: Expression) extends UnaryExpression with ImplicitCastI
     Examples:
       > SELECT _FUNC_(array(1, 2, 3), 2);
        true
-  """)
+  """,
+  group = "array_funcs")
 case class ArrayContains(left: Expression, right: Expression)
   extends BinaryExpression with ImplicitCastInputTypes {
 
@@ -1169,7 +1180,9 @@ case class ArrayContains(left: Expression, right: Expression)
     Examples:
       > SELECT _FUNC_(array(1, 2, 3), array(3, 4, 5));
        true
-  """, since = "2.4.0")
+  """,
+  group = "array_funcs",
+  since = "2.4.0")
 // scalastyle:off line.size.limit
 case class ArraysOverlap(left: Expression, right: Expression)
   extends BinaryArrayExpressionWithImplicitCast {
@@ -1392,7 +1405,9 @@ case class ArraysOverlap(left: Expression, right: Expression)
        [2,3]
       > SELECT _FUNC_(array(1, 2, 3, 4), -2, 2);
        [3,4]
-  """, since = "2.4.0")
+  """,
+  group = "array_funcs",
+  since = "2.4.0")
 // scalastyle:on line.size.limit
 case class Slice(x: Expression, start: Expression, length: Expression)
   extends TernaryExpression with ImplicitCastInputTypes {
@@ -1505,7 +1520,9 @@ case class Slice(x: Expression, start: Expression, length: Expression)
        hello world
       > SELECT _FUNC_(array('hello', null ,'world'), ' ', ',');
        hello , world
-  """, since = "2.4.0")
+  """,
+  group = "array_funcs",
+  since = "2.4.0")
 case class ArrayJoin(
     array: Expression,
     delimiter: Expression,
@@ -1668,7 +1685,9 @@ case class ArrayJoin(
     Examples:
       > SELECT _FUNC_(array(1, 20, null, 3));
        1
-  """, since = "2.4.0")
+  """,
+  group = "array_funcs",
+  since = "2.4.0")
 case class ArrayMin(child: Expression) extends UnaryExpression with ImplicitCastInputTypes {
 
   override def nullable: Boolean = true
@@ -1733,7 +1752,9 @@ case class ArrayMin(child: Expression) extends UnaryExpression with ImplicitCast
     Examples:
       > SELECT _FUNC_(array(1, 20, null, 3));
        20
-  """, since = "2.4.0")
+  """,
+  group = "array_funcs",
+  since = "2.4.0")
 case class ArrayMax(child: Expression) extends UnaryExpression with ImplicitCastInputTypes {
 
   override def nullable: Boolean = true
@@ -1807,6 +1828,7 @@ case class ArrayMax(child: Expression) extends UnaryExpression with ImplicitCast
       > SELECT _FUNC_(array(3, 2, 1), 1);
        3
   """,
+  group = "array_funcs",
   since = "2.4.0")
 case class ArrayPosition(left: Expression, right: Expression)
   extends BinaryExpression with ImplicitCastInputTypes {
@@ -2021,7 +2043,8 @@ case class ElementAt(left: Expression, right: Expression)
   """,
   note = """
     Concat logic for arrays is available since 2.4.0.
-  """)
+  """,
+  group = "array_funcs")
 case class Concat(children: Seq[Expression]) extends ComplexTypeMergingExpression {
 
   private def allowedTypes: Seq[AbstractDataType] = Seq(StringType, BinaryType, ArrayType)
@@ -2220,6 +2243,7 @@ case class Concat(children: Seq[Expression]) extends ComplexTypeMergingExpressio
       > SELECT _FUNC_(array(array(1, 2), array(3, 4)));
        [1,2,3,4]
   """,
+  group = "array_funcs",
   since = "2.4.0")
 case class Flatten(child: Expression) extends UnaryExpression {
 
@@ -2352,6 +2376,7 @@ case class Flatten(child: Expression) extends UnaryExpression {
       > SELECT _FUNC_(to_date('2018-01-01'), to_date('2018-03-01'), interval 1 month);
        [2018-01-01,2018-02-01,2018-03-01]
   """,
+  group = "array_funcs",
   since = "2.4.0"
 )
 case class Sequence(
@@ -2734,6 +2759,7 @@ object Sequence {
       > SELECT _FUNC_('123', 2);
        ["123","123"]
   """,
+  group = "array_funcs",
   since = "2.4.0")
 case class ArrayRepeat(left: Expression, right: Expression)
   extends BinaryExpression with ExpectsInputTypes {
@@ -2854,7 +2880,9 @@ case class ArrayRepeat(left: Expression, right: Expression)
     Examples:
       > SELECT _FUNC_(array(1, 2, 3, null, 3), 3);
        [1,2,null]
-  """, since = "2.4.0")
+  """,
+  group = "array_funcs",
+  since = "2.4.0")
 case class ArrayRemove(left: Expression, right: Expression)
   extends BinaryExpression with ImplicitCastInputTypes {
 
@@ -3049,7 +3077,9 @@ trait ArraySetLike {
     Examples:
       > SELECT _FUNC_(array(1, 2, 3, null, 3));
        [1,2,3,null]
-  """, since = "2.4.0")
+  """,
+  group = "array_funcs",
+  since = "2.4.0")
 case class ArrayDistinct(child: Expression)
   extends UnaryExpression with ArraySetLike with ExpectsInputTypes {
 
@@ -3226,6 +3256,7 @@ object ArrayBinaryLike {
       > SELECT _FUNC_(array(1, 2, 3), array(1, 3, 5));
        [1,2,3,5]
   """,
+  group = "array_funcs",
   since = "2.4.0")
 case class ArrayUnion(left: Expression, right: Expression) extends ArrayBinaryLike
   with ComplexTypeMergingExpression {
@@ -3437,6 +3468,7 @@ object ArrayUnion {
       > SELECT _FUNC_(array(1, 2, 3), array(1, 3, 5));
        [1,3]
   """,
+  group = "array_funcs",
   since = "2.4.0")
 case class ArrayIntersect(left: Expression, right: Expression) extends ArrayBinaryLike
   with ComplexTypeMergingExpression {
@@ -3678,6 +3710,7 @@ case class ArrayIntersect(left: Expression, right: Expression) extends ArrayBina
       > SELECT _FUNC_(array(1, 2, 3), array(1, 3, 5));
        [2]
   """,
+  group = "array_funcs",
   since = "2.4.0")
 case class ArrayExcept(left: Expression, right: Expression) extends ArrayBinaryLike
   with ComplexTypeMergingExpression {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
index 6c31511..858c91a 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
@@ -333,6 +333,7 @@ object CreateStruct extends FunctionBuilder {
       "",
       "",
       "",
+      "",
       "")
     ("struct", (info, this))
   }
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
index da1152b..4f39247 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
@@ -64,6 +64,7 @@ trait TimeZoneAwareExpression extends Expression {
  */
 @ExpressionDescription(
   usage = "_FUNC_() - Returns the current date at the start of query evaluation.",
+  group = "datetime_funcs",
   since = "1.5.0")
 case class CurrentDate(timeZoneId: Option[String] = None)
   extends LeafExpression with TimeZoneAwareExpression with CodegenFallback {
@@ -91,6 +92,7 @@ case class CurrentDate(timeZoneId: Option[String] = None)
  */
 @ExpressionDescription(
   usage = "_FUNC_() - Returns the current timestamp at the start of query evaluation.",
+  group = "datetime_funcs",
   since = "1.5.0")
 case class CurrentTimestamp() extends LeafExpression with CodegenFallback {
   override def foldable: Boolean = true
@@ -153,6 +155,7 @@ case class CurrentBatchTimestamp(
       > SELECT _FUNC_('2016-07-30', 1);
        2016-07-31
   """,
+  group = "datetime_funcs",
   since = "1.5.0")
 case class DateAdd(startDate: Expression, days: Expression)
   extends BinaryExpression with ExpectsInputTypes {
@@ -188,6 +191,7 @@ case class DateAdd(startDate: Expression, days: Expression)
       > SELECT _FUNC_('2016-07-30', 1);
        2016-07-29
   """,
+  group = "datetime_funcs",
   since = "1.5.0")
 case class DateSub(startDate: Expression, days: Expression)
   extends BinaryExpression with ExpectsInputTypes {
@@ -219,6 +223,7 @@ case class DateSub(startDate: Expression, days: Expression)
       > SELECT _FUNC_('2009-07-30 12:58:59');
        12
   """,
+  group = "datetime_funcs",
   since = "1.5.0")
 case class Hour(child: Expression, timeZoneId: Option[String] = None)
   extends UnaryExpression with TimeZoneAwareExpression with ImplicitCastInputTypes {
@@ -250,6 +255,7 @@ case class Hour(child: Expression, timeZoneId: Option[String] = None)
       > SELECT _FUNC_('2009-07-30 12:58:59');
        58
   """,
+  group = "datetime_funcs",
   since = "1.5.0")
 case class Minute(child: Expression, timeZoneId: Option[String] = None)
   extends UnaryExpression with TimeZoneAwareExpression with ImplicitCastInputTypes {
@@ -281,6 +287,7 @@ case class Minute(child: Expression, timeZoneId: Option[String] = None)
       > SELECT _FUNC_('2009-07-30 12:58:59');
        59
   """,
+  group = "datetime_funcs",
   since = "1.5.0")
 case class Second(child: Expression, timeZoneId: Option[String] = None)
   extends UnaryExpression with TimeZoneAwareExpression with ImplicitCastInputTypes {
@@ -378,6 +385,7 @@ case class Microseconds(child: Expression, timeZoneId: Option[String] = None)
       > SELECT _FUNC_('2016-04-09');
        100
   """,
+  group = "datetime_funcs",
   since = "1.5.0")
 case class DayOfYear(child: Expression) extends UnaryExpression with ImplicitCastInputTypes {
 
@@ -402,6 +410,7 @@ case class DayOfYear(child: Expression) extends UnaryExpression with ImplicitCas
       > SELECT _FUNC_('2016-07-30');
        2016
   """,
+  group = "datetime_funcs",
   since = "1.5.0")
 case class Year(child: Expression) extends UnaryExpression with ImplicitCastInputTypes {
 
@@ -442,6 +451,7 @@ case class IsoYear(child: Expression) extends UnaryExpression with ImplicitCastI
       > SELECT _FUNC_('2016-08-31');
        3
   """,
+  group = "datetime_funcs",
   since = "1.5.0")
 case class Quarter(child: Expression) extends UnaryExpression with ImplicitCastInputTypes {
 
@@ -466,6 +476,7 @@ case class Quarter(child: Expression) extends UnaryExpression with ImplicitCastI
       > SELECT _FUNC_('2016-07-30');
        7
   """,
+  group = "datetime_funcs",
   since = "1.5.0")
 case class Month(child: Expression) extends UnaryExpression with ImplicitCastInputTypes {
 
@@ -515,6 +526,7 @@ case class DayOfMonth(child: Expression) extends UnaryExpression with ImplicitCa
       > SELECT _FUNC_('2009-07-30');
        5
   """,
+  group = "datetime_funcs",
   since = "2.3.0")
 // scalastyle:on line.size.limit
 case class DayOfWeek(child: Expression) extends DayWeek {
@@ -541,6 +553,7 @@ case class DayOfWeek(child: Expression) extends DayWeek {
       > SELECT _FUNC_('2009-07-30');
        3
   """,
+  group = "datetime_funcs",
   since = "2.4.0")
 // scalastyle:on line.size.limit
 case class WeekDay(child: Expression) extends DayWeek {
@@ -574,6 +587,7 @@ abstract class DayWeek extends UnaryExpression with ImplicitCastInputTypes {
       > SELECT _FUNC_('2008-02-20');
        8
   """,
+  group = "datetime_funcs",
   since = "1.5.0")
 // scalastyle:on line.size.limit
 case class WeekOfYear(child: Expression) extends UnaryExpression with ImplicitCastInputTypes {
@@ -611,6 +625,7 @@ case class WeekOfYear(child: Expression) extends UnaryExpression with ImplicitCa
       > SELECT _FUNC_('2016-04-08', 'y');
        2016
   """,
+  group = "datetime_funcs",
   since = "1.5.0")
 // scalastyle:on line.size.limit
 case class DateFormatClass(left: Expression, right: Expression, timeZoneId: Option[String] = None)
@@ -693,6 +708,7 @@ case class DateFormatClass(left: Expression, right: Expression, timeZoneId: Opti
       > SELECT _FUNC_('2016-04-08', 'yyyy-MM-dd');
        1460098800
   """,
+  group = "datetime_funcs",
   since = "1.6.0")
 // scalastyle:on line.size.limit
 case class ToUnixTimestamp(
@@ -742,6 +758,7 @@ case class ToUnixTimestamp(
       > SELECT _FUNC_('2016-04-08', 'yyyy-MM-dd');
        1460041200
   """,
+  group = "datetime_funcs",
   since = "1.5.0")
 // scalastyle:on line.size.limit
 case class UnixTimestamp(timeExp: Expression, format: Expression, timeZoneId: Option[String] = None)
@@ -935,6 +952,7 @@ abstract class UnixTime extends ToTimestamp {
       > SELECT _FUNC_(0, 'yyyy-MM-dd HH:mm:ss');
        1969-12-31 16:00:00
   """,
+  group = "datetime_funcs",
   since = "1.5.0")
 // scalastyle:on line.size.limit
 case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[String] = None)
@@ -1055,6 +1073,7 @@ case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[
       > SELECT _FUNC_('2009-01-12');
        2009-01-31
   """,
+  group = "datetime_funcs",
   since = "1.5.0")
 case class LastDay(startDate: Expression) extends UnaryExpression with ImplicitCastInputTypes {
   override def child: Expression = startDate
@@ -1090,6 +1109,7 @@ case class LastDay(startDate: Expression) extends UnaryExpression with ImplicitC
       > SELECT _FUNC_('2015-01-14', 'TU');
        2015-01-20
   """,
+  group = "datetime_funcs",
   since = "1.5.0")
 // scalastyle:on line.size.limit
 case class NextDay(startDate: Expression, dayOfWeek: Expression)
@@ -1202,6 +1222,7 @@ case class TimeAdd(start: Expression, interval: Expression, timeZoneId: Option[S
       > SELECT _FUNC_('2016-08-31', 'Asia/Seoul');
        2016-08-31 09:00:00
   """,
+  group = "datetime_funcs",
   since = "1.5.0")
 // scalastyle:on line.size.limit
 case class FromUTCTimestamp(left: Expression, right: Expression)
@@ -1296,6 +1317,7 @@ case class TimeSub(start: Expression, interval: Expression, timeZoneId: Option[S
       > SELECT _FUNC_('2016-08-31', 1);
        2016-09-30
   """,
+  group = "datetime_funcs",
   since = "1.5.0")
 // scalastyle:on line.size.limit
 case class AddMonths(startDate: Expression, numMonths: Expression)
@@ -1345,6 +1367,7 @@ case class AddMonths(startDate: Expression, numMonths: Expression)
       > SELECT _FUNC_('1997-02-28 10:30:00', '1996-10-30', false);
        3.9495967741935485
   """,
+  group = "datetime_funcs",
   since = "1.5.0")
 // scalastyle:on line.size.limit
 case class MonthsBetween(
@@ -1406,6 +1429,7 @@ case class MonthsBetween(
       > SELECT _FUNC_('2016-08-31', 'Asia/Seoul');
        2016-08-30 15:00:00
   """,
+  group = "datetime_funcs",
   since = "1.5.0")
 // scalastyle:on line.size.limit
 case class ToUTCTimestamp(left: Expression, right: Expression)
@@ -1477,6 +1501,7 @@ case class ToUTCTimestamp(left: Expression, right: Expression)
       > SELECT _FUNC_('2016-12-31', 'yyyy-MM-dd');
        2016-12-31
   """,
+  group = "datetime_funcs",
   since = "1.5.0")
 // scalastyle:on line.size.limit
 case class ParseToDate(left: Expression, format: Option[Expression], child: Expression)
@@ -1527,6 +1552,7 @@ case class ParseToDate(left: Expression, format: Option[Expression], child: Expr
       > SELECT _FUNC_('2016-12-31', 'yyyy-MM-dd');
        2016-12-31 00:00:00
   """,
+  group = "datetime_funcs",
   since = "2.2.0")
 // scalastyle:on line.size.limit
 case class ParseToTimestamp(left: Expression, format: Option[Expression], child: Expression)
@@ -1658,6 +1684,7 @@ trait TruncInstant extends BinaryExpression with ImplicitCastInputTypes {
       > SELECT _FUNC_('1981-01-19', 'millennium');
        1001-01-01
   """,
+  group = "datetime_funcs",
   since = "1.5.0")
 // scalastyle:on line.size.limit
 case class TruncDate(date: Expression, format: Expression)
@@ -1711,6 +1738,7 @@ case class TruncDate(date: Expression, format: Expression)
       > SELECT _FUNC_('CENTURY', '2015-03-05T09:32:05.123456');
        2001-01-01 00:00:00
   """,
+  group = "datetime_funcs",
   since = "2.3.0")
 // scalastyle:on line.size.limit
 case class TruncTimestamp(
@@ -1758,6 +1786,7 @@ case class TruncTimestamp(
       > SELECT _FUNC_('2009-07-30', '2009-07-31');
        -1
   """,
+  group = "datetime_funcs",
   since = "1.5.0")
 case class DateDiff(endDate: Expression, startDate: Expression)
   extends BinaryExpression with ImplicitCastInputTypes {
@@ -1811,6 +1840,7 @@ private case class GetTimestamp(
       > SELECT _FUNC_(2019, 2, 30);
        NULL
   """,
+  group = "datetime_funcs",
   since = "3.0.0")
 case class MakeDate(year: Expression, month: Expression, day: Expression)
   extends TernaryExpression with ImplicitCastInputTypes {
@@ -1872,6 +1902,7 @@ case class MakeDate(year: Expression, month: Expression, day: Expression)
       > SELECT _FUNC_(null, 7, 22, 15, 30, 0);
        NULL
   """,
+  group = "datetime_funcs",
   since = "3.0.0")
 // scalastyle:on line.size.limit
 case class MakeTimestamp(
@@ -2179,6 +2210,7 @@ object DatePartLike {
       > SELECT _FUNC_('seconds', interval 5 hours 30 seconds 1 milliseconds 1 microseconds);
        30.001001
   """,
+  group = "datetime_funcs",
   since = "3.0.0")
 case class DatePart(field: Expression, source: Expression, child: Expression)
   extends RuntimeReplaceable {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
index 9806836..205e527 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
@@ -118,7 +118,8 @@ private[this] object SharedFactory {
     Examples:
       > SELECT _FUNC_('{"a":"b"}', '$.a');
        b
-  """)
+  """,
+  group = "json_funcs")
 case class GetJsonObject(json: Expression, path: Expression)
   extends BinaryExpression with ExpectsInputTypes with CodegenFallback {
 
@@ -341,7 +342,8 @@ case class GetJsonObject(json: Expression, path: Expression)
     Examples:
       > SELECT _FUNC_('{"a":1, "b":2}', 'a', 'b');
        1	2
-  """)
+  """,
+  group = "json_funcs")
 // scalastyle:on line.size.limit line.contains.tab
 case class JsonTuple(children: Seq[Expression])
   extends Generator with CodegenFallback {
@@ -509,6 +511,7 @@ case class JsonTuple(children: Seq[Expression])
       > SELECT _FUNC_('{"time":"26/08/2015"}', 'time Timestamp', map('timestampFormat', 'dd/MM/yyyy'));
        {"time":2015-08-26 00:00:00}
   """,
+  group = "json_funcs",
   since = "2.2.0")
 // scalastyle:on line.size.limit
 case class JsonToStructs(
@@ -628,6 +631,7 @@ case class JsonToStructs(
       > SELECT _FUNC_(array((map('a', 1))));
        [{"a":1}]
   """,
+  group = "json_funcs",
   since = "2.2.0")
 // scalastyle:on line.size.limit
 case class StructsToJson(
@@ -737,6 +741,7 @@ case class StructsToJson(
       > SELECT _FUNC_('[{"col":01}]', map('allowNumericLeadingZeros', 'true'));
        array<struct<col:bigint>>
   """,
+  group = "json_funcs",
   since = "2.4.0")
 case class SchemaOfJson(
     child: Expression,
@@ -817,6 +822,7 @@ case class SchemaOfJson(
       > SELECT _FUNC_('[1,2');
         NULL
   """,
+  group = "json_funcs",
   since = "3.1.0"
 )
 case class LengthOfJsonArray(child: Expression) extends UnaryExpression
@@ -879,13 +885,14 @@ case class LengthOfJsonArray(child: Expression) extends UnaryExpression
   """,
   examples = """
     Examples:
-      > Select _FUNC_('{}');
+      > SELECT _FUNC_('{}');
         []
-      > Select _FUNC_('{"key": "value"}');
+      > SELECT _FUNC_('{"key": "value"}');
         ["key"]
-      > Select _FUNC_('{"f1":"abc","f2":{"f3":"a", "f4":"b"}}');
+      > SELECT _FUNC_('{"f1":"abc","f2":{"f3":"a", "f4":"b"}}');
         ["f1","f2"]
   """,
+  group = "json_funcs",
   since = "3.1.0"
 )
 case class JsonObjectKeys(child: Expression) extends UnaryExpression with CodegenFallback
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala
index 9070a1a..c8b6433 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala
@@ -426,7 +426,8 @@ abstract class OffsetWindowFunction
       * default - a string expression which is to use when the offset is larger than the window.
           The default value is null.
   """,
-  since = "2.0.0")
+  since = "2.0.0",
+  group = "window_funcs")
 case class Lead(input: Expression, offset: Expression, default: Expression)
     extends OffsetWindowFunction {
 
@@ -459,7 +460,8 @@ case class Lead(input: Expression, offset: Expression, default: Expression)
       * offset - an int expression which is rows to jump back in the partition.
       * default - a string expression which is to use when the offset row does not exist.
   """,
-  since = "2.0.0")
+  since = "2.0.0",
+  group = "window_funcs")
 case class Lag(input: Expression, offset: Expression, default: Expression)
     extends OffsetWindowFunction {
 
@@ -517,7 +519,8 @@ object SizeBasedWindowFunction {
     _FUNC_() - Assigns a unique, sequential number to each row, starting with one,
       according to the ordering of rows within the window partition.
   """,
-  since = "2.0.0")
+  since = "2.0.0",
+  group = "window_funcs")
 case class RowNumber() extends RowNumberLike {
   override val evaluateExpression = rowNumber
   override def prettyName: String = "row_number"
@@ -535,7 +538,8 @@ case class RowNumber() extends RowNumberLike {
   usage = """
     _FUNC_() - Computes the position of a value relative to all values in the partition.
   """,
-  since = "2.0.0")
+  since = "2.0.0",
+  group = "window_funcs")
 case class CumeDist() extends RowNumberLike with SizeBasedWindowFunction {
   override def dataType: DataType = DoubleType
   // The frame for CUME_DIST is Range based instead of Row based, because CUME_DIST must
@@ -574,7 +578,8 @@ case class CumeDist() extends RowNumberLike with SizeBasedWindowFunction {
       * buckets - an int expression which is number of buckets to divide the rows in.
           Default value is 1.
   """,
-  since = "2.0.0")
+  since = "2.0.0",
+  group = "window_funcs")
 case class NTile(buckets: Expression) extends RowNumberLike with SizeBasedWindowFunction {
   def this() = this(Literal(1))
 
@@ -700,7 +705,8 @@ abstract class RankLike extends AggregateWindowFunction {
           trigger a change in rank. This is an internal parameter and will be assigned by the
           Analyser.
   """,
-  since = "2.0.0")
+  since = "2.0.0",
+  group = "window_funcs")
 case class Rank(children: Seq[Expression]) extends RankLike {
   def this() = this(Nil)
   override def withOrder(order: Seq[Expression]): Rank = Rank(order)
@@ -725,7 +731,8 @@ case class Rank(children: Seq[Expression]) extends RankLike {
           trigger a change in rank. This is an internal parameter and will be assigned by the
           Analyser.
   """,
-  since = "2.0.0")
+  since = "2.0.0",
+  group = "window_funcs")
 case class DenseRank(children: Seq[Expression]) extends RankLike {
   def this() = this(Nil)
   override def withOrder(order: Seq[Expression]): DenseRank = DenseRank(order)
@@ -756,7 +763,8 @@ case class DenseRank(children: Seq[Expression]) extends RankLike {
           trigger a change in rank. This is an internal parameter and will be assigned by the
           Analyser.
   """,
-  since = "2.0.0")
+  since = "2.0.0",
+  group = "window_funcs")
 case class PercentRank(children: Seq[Expression]) extends RankLike with SizeBasedWindowFunction {
   def this() = this(Nil)
   override def withOrder(order: Seq[Expression]): PercentRank = PercentRank(order)
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala
index 2d48f8d..d9c90c7 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala
@@ -345,6 +345,7 @@ object MyExtensions {
       """
        note
       """,
+      "",
       "3.0.0",
       """
        deprecated
@@ -755,6 +756,7 @@ object MyExtensions2 {
       """
        note
       """,
+      "",
       "3.0.0",
       """
        deprecated
@@ -787,6 +789,7 @@ object MyExtensions2Duplicate {
       """
        note
       """,
+      "",
       "3.0.0",
       """
        deprecated
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala
index 08f41f6..92ea0ce 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala
@@ -21,6 +21,7 @@ import java.math.BigDecimal
 
 import org.apache.spark.sql.api.java._
 import org.apache.spark.sql.catalyst.FunctionIdentifier
+import org.apache.spark.sql.catalyst.expressions.ExpressionInfo
 import org.apache.spark.sql.catalyst.plans.logical.Project
 import org.apache.spark.sql.execution.{QueryExecution, SimpleMode}
 import org.apache.spark.sql.execution.columnar.InMemoryRelation
@@ -544,6 +545,24 @@ class UDFSuite extends QueryTest with SharedSparkSession {
     assert(info.getExtended.contains("> SELECT upper('SparkSql');"))
   }
 
+  test("group info in ExpressionInfo") {
+    val info = spark.sessionState.catalog.lookupFunctionInfo(FunctionIdentifier("sum"))
+    assert(info.getGroup === "agg_funcs")
+
+    Seq("agg_funcs", "array_funcs", "datetime_funcs", "json_funcs", "map_funcs", "window_funcs")
+        .foreach { groupName =>
+      val info = new ExpressionInfo(
+        "testClass", null, "testName", null, "", "", "", groupName, "", "")
+      assert(info.getGroup === groupName)
+    }
+
+    val errMsg = intercept[IllegalArgumentException] {
+      val invalidGroupName = "invalid_group_funcs"
+      new ExpressionInfo("testClass", null, "testName", null, "", "", "", invalidGroupName, "", "")
+    }.getMessage
+    assert(errMsg.contains("'group' is malformed in the expression [testName]."))
+  }
+
   test("SPARK-28521 error message for CAST(parameter types contains DataType)") {
     val e = intercept[AnalysisException] {
       spark.sql("SELECT CAST(1)")
diff --git a/sql/create-docs.sh b/sql/create-docs.sh
index 44aa877..6614c71 100755
--- a/sql/create-docs.sh
+++ b/sql/create-docs.sh
@@ -48,6 +48,9 @@ echo "Generating SQL API Markdown files."
 echo "Generating SQL configuration table HTML file."
 "$SPARK_HOME/bin/spark-submit" gen-sql-config-docs.py
 
+echo "Generating HTML files for SQL function table and examples."
+"$SPARK_HOME/bin/spark-submit" gen-sql-functions-docs.py
+
 echo "Generating HTML files for SQL API documentation."
 mkdocs build --clean
 rm -fr docs
diff --git a/sql/gen-sql-api-docs.py b/sql/gen-sql-api-docs.py
index 4feee7a..3095a51 100644
--- a/sql/gen-sql-api-docs.py
+++ b/sql/gen-sql-api-docs.py
@@ -20,6 +20,7 @@ from collections import namedtuple
 
 from pyspark.java_gateway import launch_gateway
 
+
 ExpressionInfo = namedtuple(
     "ExpressionInfo", "className name usage arguments examples note since deprecated")
 
@@ -159,7 +160,7 @@ def _make_pretty_deprecated(deprecated):
         return "**Deprecated:**\n%s\n" % deprecated
 
 
-def generate_sql_markdown(jvm, path):
+def generate_sql_api_markdown(jvm, path):
     """
     Generates a markdown file after listing the function information. The output file
     is created in `path`.
@@ -223,4 +224,4 @@ if __name__ == "__main__":
     jvm = launch_gateway().jvm
     spark_root_dir = os.path.dirname(os.path.dirname(__file__))
     markdown_file_path = os.path.join(spark_root_dir, "sql/docs/index.md")
-    generate_sql_markdown(jvm, markdown_file_path)
+    generate_sql_api_markdown(jvm, markdown_file_path)
diff --git a/sql/gen-sql-config-docs.py b/sql/gen-sql-config-docs.py
index 98212ad..0043c41 100644
--- a/sql/gen-sql-config-docs.py
+++ b/sql/gen-sql-config-docs.py
@@ -22,8 +22,10 @@ from textwrap import dedent
 
 # To avoid adding a new direct dependency, we import markdown from within mkdocs.
 from mkdocs.structure.pages import markdown
+
 from pyspark.java_gateway import launch_gateway
 
+
 SQLConfEntry = namedtuple(
     "SQLConfEntry", ["name", "default", "description", "version"])
 
@@ -41,7 +43,7 @@ def get_public_sql_configs(jvm):
     return sql_configs
 
 
-def generate_sql_configs_table(sql_configs, path):
+def generate_sql_configs_table_html(sql_configs, path):
     """
     Generates an HTML table at `path` that lists all public SQL
     configuration options.
@@ -116,6 +118,7 @@ if __name__ == "__main__":
     sql_configs = get_public_sql_configs(jvm)
 
     spark_root_dir = os.path.dirname(os.path.dirname(__file__))
-    sql_configs_table_path = os.path.join(spark_root_dir, "docs/sql-configs.html")
+    sql_configs_table_path = os.path.join(
+        spark_root_dir, "docs/generated-sql-configuration-table.html")
 
-    generate_sql_configs_table(sql_configs, path=sql_configs_table_path)
+    generate_sql_configs_table_html(sql_configs, path=sql_configs_table_path)
diff --git a/sql/gen-sql-functions-docs.py b/sql/gen-sql-functions-docs.py
new file mode 100644
index 0000000..7f0b2ae
--- /dev/null
+++ b/sql/gen-sql-functions-docs.py
@@ -0,0 +1,228 @@
+#
+# 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.
+#
+
+import itertools
+import os
+import re
+from collections import namedtuple
+
+# To avoid adding a new direct dependency, we import markdown from within mkdocs.
+from mkdocs.structure.pages import markdown
+
+from pyspark.java_gateway import launch_gateway
+
+
+ExpressionInfo = namedtuple("ExpressionInfo", "name usage examples group")
+
+groups = {
+    "agg_funcs", "array_funcs", "datetime_funcs",
+    "json_funcs", "map_funcs", "window_funcs",
+}
+
+
+def _list_grouped_function_infos(jvm):
+    """
+    Returns a list of function information grouped by each group value via JVM.
+    Sorts wrapped expression infos in each group by name and returns them.
+    """
+
+    jinfos = jvm.org.apache.spark.sql.api.python.PythonSQLUtils.listBuiltinFunctionInfos()
+    infos = []
+
+    for jinfo in filter(lambda x: x.getGroup() in groups, jinfos):
+        name = jinfo.getName()
+        usage = jinfo.getUsage()
+        usage = usage.replace("_FUNC_", name) if usage is not None else usage
+        infos.append(ExpressionInfo(
+            name=name,
+            usage=usage,
+            examples=jinfo.getExamples().replace("_FUNC_", name),
+            group=jinfo.getGroup()))
+
+    # Groups expression info by each group value
+    grouped_infos = itertools.groupby(sorted(infos, key=lambda x: x.group), key=lambda x: x.group)
+    # Then, sort expression infos in each group by name
+    return [(k, sorted(g, key=lambda x: x.name)) for k, g in grouped_infos]
+
+
+# TODO(SPARK-31499): Needs to add a column to describe arguments and their types
+def _make_pretty_usage(infos):
+    """
+    Makes the usage description pretty and returns a formatted string.
+
+    Expected input:
+
+        func(*) - ...
+
+        func(expr[, expr...]) - ...
+
+    Expected output:
+    <table class="table">
+      <thead>
+        <tr>
+          <th style="width:25%">Function</th>
+          <th>Description</th>
+        </tr>
+      </thead>
+      <tbody>
+        <tr>
+          <td>func(*)</td>
+          <td>...</td>
+        </tr>
+        <tr>
+          <td>func(expr[, expr...])</td>
+          <td>...</td>
+        </tr>
+      </tbody>
+      ...
+    </table>
+
+    """
+
+    result = []
+    result.append("<table class=\"table\">")
+    result.append("  <thead>")
+    result.append("    <tr>")
+    result.append("      <th style=\"width:25%\">Function</th>")
+    result.append("      <th>Description</th>")
+    result.append("    </tr>")
+    result.append("  </thead>")
+    result.append("  <tbody>")
+
+    for info in infos:
+        # Extracts (signature, description) pairs from `info.usage`, e.g.,
+        # the signature is `func(expr)` and the description is `...` in an usage `func(expr) - ...`.
+        usages = iter(re.split(r"(%s\(.*\)) - " % info.name, info.usage.strip())[1:])
+        for (sig, description) in zip(usages, usages):
+            result.append("    <tr>")
+            result.append("      <td>%s</td>" % sig)
+            result.append("      <td>%s</td>" % description.strip())
+            result.append("    </tr>")
+
+    result.append("  </tbody>")
+    result.append("</table>\n")
+    return "\n".join(result)
+
+
+def _make_pretty_examples(jspark, infos):
+    """
+    Makes the examples description pretty and returns a formatted string if `infos`
+    has any `examples` starting with the example prefix. Otherwise, returns None.
+
+    Expected input:
+
+        Examples:
+          > SELECT func(col)...;
+           ...
+          > SELECT func(col)...;
+           ...
+
+    Expected output:
+    <div class="codehilite"><pre><span></span>
+      <span class="c1">-- func</span>
+      <span class="k">SELECT</span>
+      ...
+    </pre></div>
+    ```
+
+    """
+
+    pretty_output = ""
+    for info in infos:
+        if info.examples.startswith("\n    Examples:"):
+            output = []
+            output.append("-- %s" % info.name)
+            query_examples = filter(lambda x: x.startswith("      > "), info.examples.split("\n"))
+            for query_example in query_examples:
+                query = query_example.lstrip("      > ")
+                print("    %s" % query)
+                query_output = jspark.sql(query).showString(20, 20, False)
+                output.append(query)
+                output.append(query_output)
+            pretty_output += "\n" + "\n".join(output)
+    if pretty_output != "":
+        return markdown.markdown(
+            "```sql%s```" % pretty_output, extensions=['codehilite', 'fenced_code'])
+
+
+def generate_functions_table_html(jvm, html_output_dir):
+    """
+    Generates a HTML file after listing the function information. The output file
+    is created under `html_output_dir`.
+
+    Expected output:
+
+    <table class="table">
+      <thead>
+        <tr>
+          <th style="width:25%">Function</th>
+          <th>Description</th>
+        </tr>
+      </thead>
+      <tbody>
+        <tr>
+          <td>func(*)</td>
+          <td>...</td>
+        </tr>
+        <tr>
+          <td>func(expr[, expr...])</td>
+          <td>...</td>
+        </tr>
+      </tbody>
+      ...
+    </table>
+
+    """
+    for key, infos in _list_grouped_function_infos(jvm):
+        function_table = _make_pretty_usage(infos)
+        key = key.replace("_", "-")
+        with open("%s/generated-%s-table.html" % (html_output_dir, key), 'w') as table_html:
+            table_html.write(function_table)
+
+
+def generate_functions_examples_html(jvm, jspark, html_output_dir):
+    """
+    Generates a HTML file after listing and executing the function information.
+    The output file is created under `html_output_dir`.
+
+    Expected output:
+
+    <div class="codehilite"><pre><span></span>
+      <span class="c1">-- func</span>
+      <span class="k">SELECT</span>
+      ...
+    </pre></div>
+
+    """
+    print("Running SQL examples to generate formatted output.")
+    for key, infos in _list_grouped_function_infos(jvm):
+        examples = _make_pretty_examples(jspark, infos)
+        key = key.replace("_", "-")
+        if examples is not None:
+            with open("%s/generated-%s-examples.html" % (
+                    html_output_dir, key), 'w') as examples_html:
+                examples_html.write(examples)
+
+
+if __name__ == "__main__":
+    jvm = launch_gateway().jvm
+    jspark = jvm.org.apache.spark.sql.SparkSession.builder().getOrCreate()
+    jspark.sparkContext().setLogLevel("ERROR")  # Make it less noisy.
+    spark_root_dir = os.path.dirname(os.path.dirname(__file__))
+    html_output_dir = os.path.join(spark_root_dir, "docs")
+    generate_functions_table_html(jvm, html_output_dir)
+    generate_functions_examples_html(jvm, jspark, html_output_dir)


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