Skip to content

Commit

Permalink
[SPARK-31429][SQL][DOC] Automatically generates a SQL document for bu…
Browse files Browse the repository at this point in the history
…ilt-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 apache#28224 from maropu/SPARK-31429.

Lead-authored-by: Takeshi Yamamuro <[email protected]>
Co-authored-by: HyukjinKwon <[email protected]>
Signed-off-by: HyukjinKwon <[email protected]>
  • Loading branch information
maropu and HyukjinKwon committed Apr 21, 2020
1 parent 4f8b03d commit e42dbe7
Show file tree
Hide file tree
Showing 38 changed files with 528 additions and 42 deletions.
2 changes: 1 addition & 1 deletion docs/.gitignore
Original file line number Diff line number Diff line change
@@ -1 +1 @@
sql-configs.html
generated-*.html
2 changes: 2 additions & 0 deletions docs/_data/menu-sql.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
4 changes: 2 additions & 2 deletions docs/configuration.md
Original file line number Diff line number Diff line change
Expand Up @@ -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 %}
Expand Down
77 changes: 77 additions & 0 deletions docs/sql-ref-functions-builtin.md
Original file line number Diff line number Diff line change
@@ -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 %}

12 changes: 12 additions & 0 deletions docs/sql-ref-functions.md
Original file line number Diff line number Diff line change
Expand Up @@ -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, UDAFs and UDTFs.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,21 +31,24 @@
* `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.
*
* `examples()` describes examples for the expression.
*
* `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>
Expand All @@ -68,6 +71,7 @@
* note = """
* ...
* """,
* group = "agg_funcs",
* since = "3.0.0",
* deprecated = """
* ...
Expand All @@ -78,8 +82,9 @@
* 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.
Expand All @@ -98,6 +103,7 @@
String arguments() default "";
String examples() default "";
String note() default "";
String group() default "";
String since() default "";
String deprecated() default "";
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,10 @@

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.
*/
Expand All @@ -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;
}
Expand Down Expand Up @@ -75,6 +84,10 @@ public String getDeprecated() {
return deprecated;
}

public String getGroup() {
return group;
}

public String getDb() {
return db;
}
Expand All @@ -87,13 +100,15 @@ public ExpressionInfo(
String arguments,
String examples,
String note,
String group,
String since,
String deprecated) {
assert name != null;
assert arguments != null;
assert examples != null;
assert examples.isEmpty() || examples.contains(" Examples:");
assert note != null;
assert group != null;
assert since != null;
assert deprecated != null;

Expand All @@ -104,6 +119,7 @@ public ExpressionInfo(
this.arguments = arguments;
this.examples = examples;
this.note = note;
this.group = group;
this.since = since;
this.deprecated = deprecated;

Expand All @@ -120,6 +136,11 @@ public 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 [" +
Expand All @@ -140,11 +161,11 @@ public 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, "", "", "", "", "", "");
}

/**
Expand All @@ -155,7 +176,7 @@ public ExpressionInfo(String className, String db, String name) {
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) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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))
}

Expand All @@ -675,6 +675,7 @@ object FunctionRegistry {
df.arguments(),
df.examples(),
df.note(),
df.group(),
df.since(),
df.deprecated())
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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 {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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) {
Expand All @@ -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) {
Expand All @@ -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) {

Expand All @@ -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) {

Expand All @@ -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) {

Expand All @@ -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) {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand Down
Loading

0 comments on commit e42dbe7

Please sign in to comment.