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

wenchen pushed a commit to branch branch-3.0
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.0 by this push:
     new 75fc384  [SPARK-31372][SQL][TEST] Display expression schema for double 
check
75fc384 is described below

commit 75fc3840b704306fbf64d059898b854e8c46caf9
Author: beliefer <[email protected]>
AuthorDate: Thu Apr 30 03:58:04 2020 +0000

    [SPARK-31372][SQL][TEST] Display expression schema for double check
    
    ### What changes were proposed in this pull request?
    Although SPARK-30184 Implement a helper method for aliasing functions, 
developers always forget to using this improvement.
    We need to add more powerful guarantees so that aliases outputed by 
built-in functions are correct.
    This PR extracts the SQL from the example of expressions, and output the 
SQL and its schema into one golden file.
    By checking the golden file, we can find the expressions whose aliases are 
not displayed correctly, and then fix them.
    
    ### Why are the changes needed?
    Ensure that the output alias is correct
    
    ### Does this PR introduce any user-facing change?
    'No'.
    
    ### How was this patch tested?
    Jenkins test.
    
    Closes #28194 from beliefer/check-expression-schema.
    
    Lead-authored-by: beliefer <[email protected]>
    Co-authored-by: gengjiaan <[email protected]>
    Signed-off-by: Wenchen Fan <[email protected]>
    (cherry picked from commit 1d1bb79bc695dbaa00699e6fc9073233b60ed395)
    Signed-off-by: Wenchen Fan <[email protected]>
---
 .../sql-functions/sql-expression-schema.md         | 341 +++++++++++++++++++++
 .../apache/spark/sql/ExpressionsSchemaSuite.scala  | 187 +++++++++++
 2 files changed, 528 insertions(+)

diff --git a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md 
b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md
new file mode 100644
index 0000000..1e22ae2
--- /dev/null
+++ b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md
@@ -0,0 +1,341 @@
+<!-- Automatically generated byExpressionsSchemaSuite -->
+## Summary
+  - Number of queries: 333
+  - Number of expressions that missing example: 34
+  - Expressions missing examples: 
and,string,tinyint,double,smallint,date,decimal,boolean,float,binary,bigint,int,timestamp,cume_dist,dense_rank,input_file_block_length,input_file_block_start,input_file_name,lag,lead,monotonically_increasing_id,ntile,struct,!,not,or,percent_rank,rank,row_number,spark_partition_id,version,window,positive,count_min_sketch
+## Schema of Built-in Functions
+| Class name | Function name or alias | Query example | Output schema |
+| ---------- | ---------------------- | ------------- | ------------- |
+| org.apache.spark.sql.catalyst.expressions.Abs | abs | SELECT abs(-1) | 
struct<abs(-1):int> |
+| org.apache.spark.sql.catalyst.expressions.Acos | acos | SELECT acos(1) | 
struct<ACOS(CAST(1 AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.Acosh | acosh | SELECT acosh(1) | 
struct<ACOSH(CAST(1 AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.Add | + | SELECT 1 + 2 | struct<(1 
+ 2):int> |
+| org.apache.spark.sql.catalyst.expressions.AddMonths | add_months | SELECT 
add_months('2016-08-31', 1) | struct<add_months(CAST(2016-08-31 AS DATE), 
1):date> |
+| org.apache.spark.sql.catalyst.expressions.And | and | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.ArrayAggregate | aggregate | 
SELECT aggregate(array(1, 2, 3), 0, (acc, x) -> acc + x) | 
struct<aggregate(array(1, 2, 3), 0, lambdafunction((namedlambdavariable() + 
namedlambdavariable()), namedlambdavariable(), namedlambdavariable()), 
lambdafunction(namedlambdavariable(), namedlambdavariable())):int> |
+| org.apache.spark.sql.catalyst.expressions.ArrayContains | array_contains | 
SELECT array_contains(array(1, 2, 3), 2) | struct<array_contains(array(1, 2, 
3), 2):boolean> |
+| org.apache.spark.sql.catalyst.expressions.ArrayDistinct | array_distinct | 
SELECT array_distinct(array(1, 2, 3, null, 3)) | struct<array_distinct(array(1, 
2, 3, CAST(NULL AS INT), 3)):array<int>> |
+| org.apache.spark.sql.catalyst.expressions.ArrayExcept | array_except | 
SELECT array_except(array(1, 2, 3), array(1, 3, 5)) | 
struct<array_except(array(1, 2, 3), array(1, 3, 5)):array<int>> |
+| org.apache.spark.sql.catalyst.expressions.ArrayExists | exists | SELECT 
exists(array(1, 2, 3), x -> x % 2 == 0) | struct<exists(array(1, 2, 3), 
lambdafunction(((namedlambdavariable() % 2) = 0), 
namedlambdavariable())):boolean> |
+| org.apache.spark.sql.catalyst.expressions.ArrayFilter | filter | SELECT 
filter(array(1, 2, 3), x -> x % 2 == 1) | struct<filter(array(1, 2, 3), 
lambdafunction(((namedlambdavariable() % 2) = 1), 
namedlambdavariable())):array<int>> |
+| org.apache.spark.sql.catalyst.expressions.ArrayForAll | forall | SELECT 
forall(array(1, 2, 3), x -> x % 2 == 0) | struct<forall(array(1, 2, 3), 
lambdafunction(((namedlambdavariable() % 2) = 0), 
namedlambdavariable())):boolean> |
+| org.apache.spark.sql.catalyst.expressions.ArrayIntersect | array_intersect | 
SELECT array_intersect(array(1, 2, 3), array(1, 3, 5)) | 
struct<array_intersect(array(1, 2, 3), array(1, 3, 5)):array<int>> |
+| org.apache.spark.sql.catalyst.expressions.ArrayJoin | array_join | SELECT 
array_join(array('hello', 'world'), ' ') | struct<array_join(array(hello, 
world),  ):string> |
+| org.apache.spark.sql.catalyst.expressions.ArrayMax | array_max | SELECT 
array_max(array(1, 20, null, 3)) | struct<array_max(array(1, 20, CAST(NULL AS 
INT), 3)):int> |
+| org.apache.spark.sql.catalyst.expressions.ArrayMin | array_min | SELECT 
array_min(array(1, 20, null, 3)) | struct<array_min(array(1, 20, CAST(NULL AS 
INT), 3)):int> |
+| org.apache.spark.sql.catalyst.expressions.ArrayPosition | array_position | 
SELECT array_position(array(3, 2, 1), 1) | struct<array_position(array(3, 2, 
1), 1):bigint> |
+| org.apache.spark.sql.catalyst.expressions.ArrayRemove | array_remove | 
SELECT array_remove(array(1, 2, 3, null, 3), 3) | struct<array_remove(array(1, 
2, 3, CAST(NULL AS INT), 3), 3):array<int>> |
+| org.apache.spark.sql.catalyst.expressions.ArrayRepeat | array_repeat | 
SELECT array_repeat('123', 2) | struct<array_repeat(123, 2):array<string>> |
+| org.apache.spark.sql.catalyst.expressions.ArraySort | array_sort | SELECT 
array_sort(array(5, 6, 1), (left, right) -> case when left < right then -1 when 
left > right then 1 else 0 end) | struct<array_sort(array(5, 6, 1), 
lambdafunction(CASE WHEN (namedlambdavariable() < namedlambdavariable()) THEN 
-1 WHEN (namedlambdavariable() > namedlambdavariable()) THEN 1 ELSE 0 END, 
namedlambdavariable(), namedlambdavariable())):array<int>> |
+| org.apache.spark.sql.catalyst.expressions.ArrayTransform | transform | 
SELECT transform(array(1, 2, 3), x -> x + 1) | struct<transform(array(1, 2, 3), 
lambdafunction((namedlambdavariable() + 1), namedlambdavariable())):array<int>> 
|
+| org.apache.spark.sql.catalyst.expressions.ArrayUnion | array_union | SELECT 
array_union(array(1, 2, 3), array(1, 3, 5)) | struct<array_union(array(1, 2, 
3), array(1, 3, 5)):array<int>> |
+| org.apache.spark.sql.catalyst.expressions.ArraysOverlap | arrays_overlap | 
SELECT arrays_overlap(array(1, 2, 3), array(3, 4, 5)) | 
struct<arrays_overlap(array(1, 2, 3), array(3, 4, 5)):boolean> |
+| org.apache.spark.sql.catalyst.expressions.ArraysZip | arrays_zip | SELECT 
arrays_zip(array(1, 2, 3), array(2, 3, 4)) | struct<arrays_zip(array(1, 2, 3), 
array(2, 3, 4)):array<struct<0:int,1:int>>> |
+| org.apache.spark.sql.catalyst.expressions.Ascii | ascii | SELECT 
ascii('222') | struct<ascii(222):int> |
+| org.apache.spark.sql.catalyst.expressions.Asin | asin | SELECT asin(0) | 
struct<ASIN(CAST(0 AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.Asinh | asinh | SELECT asinh(0) | 
struct<ASINH(CAST(0 AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.AssertTrue | assert_true | SELECT 
assert_true(0 < 1) | struct<assert_true((0 < 1)):null> |
+| org.apache.spark.sql.catalyst.expressions.Atan | atan | SELECT atan(0) | 
struct<ATAN(CAST(0 AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.Atan2 | atan2 | SELECT atan2(0, 0) 
| struct<ATAN2(CAST(0 AS DOUBLE), CAST(0 AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.Atanh | atanh | SELECT atanh(0) | 
struct<ATANH(CAST(0 AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.BRound | bround | SELECT 
bround(2.5, 0) | struct<bround(2.5, 0):decimal(2,0)> |
+| org.apache.spark.sql.catalyst.expressions.Base64 | base64 | SELECT 
base64('Spark SQL') | struct<base64(CAST(Spark SQL AS BINARY)):string> |
+| org.apache.spark.sql.catalyst.expressions.Bin | bin | SELECT bin(13) | 
struct<bin(CAST(13 AS BIGINT)):string> |
+| org.apache.spark.sql.catalyst.expressions.BitLength | bit_length | SELECT 
bit_length('Spark SQL') | struct<bit_length(Spark SQL):int> |
+| org.apache.spark.sql.catalyst.expressions.BitwiseAnd | & | SELECT 3 & 5 | 
struct<(3 & 5):int> |
+| org.apache.spark.sql.catalyst.expressions.BitwiseCount | bit_count | SELECT 
bit_count(0) | struct<bit_count(0):int> |
+| org.apache.spark.sql.catalyst.expressions.BitwiseNot | ~ | SELECT ~ 0 | 
struct<~0:int> |
+| org.apache.spark.sql.catalyst.expressions.BitwiseOr | &#124; | SELECT 3 
&#124; 5 | struct<(3 &#124; 5):int> |
+| org.apache.spark.sql.catalyst.expressions.BitwiseXor | ^ | SELECT 3 ^ 5 | 
struct<(3 ^ 5):int> |
+| org.apache.spark.sql.catalyst.expressions.CallMethodViaReflection | 
java_method | SELECT java_method('java.util.UUID', 'randomUUID') | 
struct<java_method(java.util.UUID, randomUUID):string> |
+| org.apache.spark.sql.catalyst.expressions.CallMethodViaReflection | reflect 
| SELECT reflect('java.util.UUID', 'randomUUID') | 
struct<reflect(java.util.UUID, randomUUID):string> |
+| org.apache.spark.sql.catalyst.expressions.CaseWhen | when | SELECT CASE WHEN 
1 > 0 THEN 1 WHEN 2 > 0 THEN 2.0 ELSE 1.2 END | struct<CASE WHEN (1 > 0) THEN 
CAST(1 AS DECIMAL(11,1)) WHEN (2 > 0) THEN CAST(2.0 AS DECIMAL(11,1)) ELSE 
CAST(1.2 AS DECIMAL(11,1)) END:decimal(11,1)> |
+| org.apache.spark.sql.catalyst.expressions.Cast | string | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.Cast | cast | SELECT cast('10' as 
int) | struct<CAST(10 AS INT):int> |
+| org.apache.spark.sql.catalyst.expressions.Cast | tinyint | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.Cast | double | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.Cast | smallint | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.Cast | date | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.Cast | decimal | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.Cast | boolean | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.Cast | float | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.Cast | binary | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.Cast | bigint | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.Cast | int | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.Cast | timestamp | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.Cbrt | cbrt | SELECT cbrt(27.0) | 
struct<CBRT(CAST(27.0 AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.Ceil | ceil | SELECT ceil(-0.1) | 
struct<CEIL(-0.1):decimal(1,0)> |
+| org.apache.spark.sql.catalyst.expressions.Ceil | ceiling | SELECT 
ceiling(-0.1) | struct<ceiling(-0.1):decimal(1,0)> |
+| org.apache.spark.sql.catalyst.expressions.Chr | char | SELECT char(65) | 
struct<char(CAST(65 AS BIGINT)):string> |
+| org.apache.spark.sql.catalyst.expressions.Chr | chr | SELECT chr(65) | 
struct<chr(CAST(65 AS BIGINT)):string> |
+| org.apache.spark.sql.catalyst.expressions.Coalesce | coalesce | SELECT 
coalesce(NULL, 1, NULL) | struct<coalesce(CAST(NULL AS INT), 1, CAST(NULL AS 
INT)):int> |
+| org.apache.spark.sql.catalyst.expressions.Concat | concat | SELECT 
concat('Spark', 'SQL') | struct<concat(Spark, SQL):string> |
+| org.apache.spark.sql.catalyst.expressions.ConcatWs | concat_ws | SELECT 
concat_ws(' ', 'Spark', 'SQL') | struct<concat_ws( , Spark, SQL):string> |
+| org.apache.spark.sql.catalyst.expressions.Conv | conv | SELECT conv('100', 
2, 10) | struct<conv(100, 2, 10):string> |
+| org.apache.spark.sql.catalyst.expressions.Cos | cos | SELECT cos(0) | 
struct<COS(CAST(0 AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.Cosh | cosh | SELECT cosh(0) | 
struct<COSH(CAST(0 AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.Cot | cot | SELECT cot(1) | 
struct<COT(CAST(1 AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.Crc32 | crc32 | SELECT 
crc32('Spark') | struct<crc32(CAST(Spark AS BINARY)):bigint> |
+| org.apache.spark.sql.catalyst.expressions.CreateArray | array | SELECT 
array(1, 2, 3) | struct<array(1, 2, 3):array<int>> |
+| org.apache.spark.sql.catalyst.expressions.CreateMap | map | SELECT map(1.0, 
'2', 3.0, '4') | struct<map(1.0, 2, 3.0, 4):map<decimal(2,1),string>> |
+| org.apache.spark.sql.catalyst.expressions.CreateNamedStruct | named_struct | 
SELECT named_struct("a", 1, "b", 2, "c", 3) | struct<named_struct(a, 1, b, 2, 
c, 3):struct<a:int,b:int,c:int>> |
+| org.apache.spark.sql.catalyst.expressions.CsvToStructs | from_csv | SELECT 
from_csv('1, 0.8', 'a INT, b DOUBLE') | struct<from_csv(1, 
0.8):struct<a:int,b:double>> |
+| org.apache.spark.sql.catalyst.expressions.Cube | cube | SELECT name, age, 
count(*) FROM VALUES (2, 'Alice'), (5, 'Bob') people(age, name) GROUP BY 
cube(name, age) | struct<name:string,age:int,count(1):bigint> |
+| org.apache.spark.sql.catalyst.expressions.CumeDist | cume_dist | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.CurrentDatabase | current_database 
| SELECT current_database() | struct<current_database():string> |
+| org.apache.spark.sql.catalyst.expressions.CurrentDate | current_date | 
SELECT current_date() | struct<current_date():date> |
+| org.apache.spark.sql.catalyst.expressions.CurrentTimestamp | 
current_timestamp | SELECT current_timestamp() | 
struct<current_timestamp():timestamp> |
+| org.apache.spark.sql.catalyst.expressions.DateAdd | date_add | SELECT 
date_add('2016-07-30', 1) | struct<date_add(CAST(2016-07-30 AS DATE), 1):date> |
+| org.apache.spark.sql.catalyst.expressions.DateDiff | datediff | SELECT 
datediff('2009-07-31', '2009-07-30') | struct<datediff(CAST(2009-07-31 AS 
DATE), CAST(2009-07-30 AS DATE)):int> |
+| org.apache.spark.sql.catalyst.expressions.DateFormatClass | date_format | 
SELECT date_format('2016-04-08', 'y') | struct<date_format(CAST(2016-04-08 AS 
TIMESTAMP), y):string> |
+| org.apache.spark.sql.catalyst.expressions.DatePart | date_part | SELECT 
date_part('YEAR', TIMESTAMP '2019-08-12 01:00:00.123456') | 
struct<date_part('YEAR', TIMESTAMP '2019-08-12 01:00:00.123456'):int> |
+| org.apache.spark.sql.catalyst.expressions.DateSub | date_sub | SELECT 
date_sub('2016-07-30', 1) | struct<date_sub(CAST(2016-07-30 AS DATE), 1):date> |
+| org.apache.spark.sql.catalyst.expressions.DayOfMonth | day | SELECT 
day('2009-07-30') | struct<day(CAST(2009-07-30 AS DATE)):int> |
+| org.apache.spark.sql.catalyst.expressions.DayOfMonth | dayofmonth | SELECT 
dayofmonth('2009-07-30') | struct<dayofmonth(CAST(2009-07-30 AS DATE)):int> |
+| org.apache.spark.sql.catalyst.expressions.DayOfWeek | dayofweek | SELECT 
dayofweek('2009-07-30') | struct<dayofweek(CAST(2009-07-30 AS DATE)):int> |
+| org.apache.spark.sql.catalyst.expressions.DayOfYear | dayofyear | SELECT 
dayofyear('2016-04-09') | struct<dayofyear(CAST(2016-04-09 AS DATE)):int> |
+| org.apache.spark.sql.catalyst.expressions.Decode | decode | SELECT 
decode(encode('abc', 'utf-8'), 'utf-8') | struct<decode(encode(abc, utf-8), 
utf-8):string> |
+| org.apache.spark.sql.catalyst.expressions.DenseRank | dense_rank | N/A | N/A 
|
+| org.apache.spark.sql.catalyst.expressions.Divide | / | SELECT 3 / 2 | 
struct<(CAST(3 AS DOUBLE) / CAST(2 AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.ElementAt | element_at | SELECT 
element_at(array(1, 2, 3), 2) | struct<element_at(array(1, 2, 3), 2):int> |
+| org.apache.spark.sql.catalyst.expressions.Elt | elt | SELECT elt(1, 'scala', 
'java') | struct<elt(1, scala, java):string> |
+| org.apache.spark.sql.catalyst.expressions.Encode | encode | SELECT 
encode('abc', 'utf-8') | struct<encode(abc, utf-8):binary> |
+| org.apache.spark.sql.catalyst.expressions.EqualNullSafe | <=> | SELECT 2 <=> 
2 | struct<(2 <=> 2):boolean> |
+| org.apache.spark.sql.catalyst.expressions.EqualTo | = | SELECT 2 = 2 | 
struct<(2 = 2):boolean> |
+| org.apache.spark.sql.catalyst.expressions.EqualTo | == | SELECT 2 == 2 | 
struct<(2 = 2):boolean> |
+| org.apache.spark.sql.catalyst.expressions.EulerNumber | e | SELECT e() | 
struct<E():double> |
+| org.apache.spark.sql.catalyst.expressions.Exp | exp | SELECT exp(0) | 
struct<EXP(CAST(0 AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.Explode | explode | SELECT 
explode(array(10, 20)) | struct<col:int> |
+| org.apache.spark.sql.catalyst.expressions.Explode | explode_outer | SELECT 
explode_outer(array(10, 20)) | struct<col:int> |
+| org.apache.spark.sql.catalyst.expressions.Expm1 | expm1 | SELECT expm1(0) | 
struct<EXPM1(CAST(0 AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.Extract | extract | SELECT 
extract(YEAR FROM TIMESTAMP '2019-08-12 01:00:00.123456') | 
struct<extract('YEAR' FROM TIMESTAMP '2019-08-12 01:00:00.123456'):int> |
+| org.apache.spark.sql.catalyst.expressions.Factorial | factorial | SELECT 
factorial(5) | struct<factorial(5):bigint> |
+| org.apache.spark.sql.catalyst.expressions.FindInSet | find_in_set | SELECT 
find_in_set('ab','abc,b,ab,c,def') | struct<find_in_set(ab, 
abc,b,ab,c,def):int> |
+| org.apache.spark.sql.catalyst.expressions.Flatten | flatten | SELECT 
flatten(array(array(1, 2), array(3, 4))) | struct<flatten(array(array(1, 2), 
array(3, 4))):array<int>> |
+| org.apache.spark.sql.catalyst.expressions.Floor | floor | SELECT floor(-0.1) 
| struct<FLOOR(-0.1):decimal(1,0)> |
+| org.apache.spark.sql.catalyst.expressions.FormatNumber | format_number | 
SELECT format_number(12332.123456, 4) | struct<format_number(12332.123456, 
4):string> |
+| org.apache.spark.sql.catalyst.expressions.FormatString | printf | SELECT 
printf("Hello World %d %s", 100, "days") | struct<printf(Hello World %d %s, 
100, days):string> |
+| org.apache.spark.sql.catalyst.expressions.FormatString | format_string | 
SELECT format_string("Hello World %d %s", 100, "days") | 
struct<format_string(Hello World %d %s, 100, days):string> |
+| org.apache.spark.sql.catalyst.expressions.FromUTCTimestamp | 
from_utc_timestamp | SELECT from_utc_timestamp('2016-08-31', 'Asia/Seoul') | 
struct<from_utc_timestamp(CAST(2016-08-31 AS TIMESTAMP), Asia/Seoul):timestamp> 
|
+| org.apache.spark.sql.catalyst.expressions.FromUnixTime | from_unixtime | 
SELECT from_unixtime(0, 'yyyy-MM-dd HH:mm:ss') | struct<from_unixtime(CAST(0 AS 
BIGINT), yyyy-MM-dd HH:mm:ss):string> |
+| org.apache.spark.sql.catalyst.expressions.GetJsonObject | get_json_object | 
SELECT get_json_object('{"a":"b"}', '$.a') | struct<get_json_object({"a":"b"}, 
$.a):string> |
+| org.apache.spark.sql.catalyst.expressions.GreaterThan | > | SELECT 2 > 1 | 
struct<(2 > 1):boolean> |
+| org.apache.spark.sql.catalyst.expressions.GreaterThanOrEqual | >= | SELECT 2 
>= 1 | struct<(2 >= 1):boolean> |
+| org.apache.spark.sql.catalyst.expressions.Greatest | greatest | SELECT 
greatest(10, 9, 2, 4, 3) | struct<greatest(10, 9, 2, 4, 3):int> |
+| org.apache.spark.sql.catalyst.expressions.Grouping | grouping | SELECT name, 
grouping(name), sum(age) FROM VALUES (2, 'Alice'), (5, 'Bob') people(age, name) 
GROUP BY cube(name) | 
struct<name:string,grouping(name):tinyint,sum(age):bigint> |
+| org.apache.spark.sql.catalyst.expressions.GroupingID | grouping_id | SELECT 
name, grouping_id(), sum(age), avg(height) FROM VALUES (2, 'Alice', 165), (5, 
'Bob', 180) people(age, name, height) GROUP BY cube(name, height) | 
struct<name:string,grouping_id():bigint,sum(age):bigint,avg(height):double> |
+| org.apache.spark.sql.catalyst.expressions.Hex | hex | SELECT hex(17) | 
struct<hex(CAST(17 AS BIGINT)):string> |
+| org.apache.spark.sql.catalyst.expressions.Hour | hour | SELECT 
hour('2009-07-30 12:58:59') | struct<hour(CAST(2009-07-30 12:58:59 AS 
TIMESTAMP)):int> |
+| org.apache.spark.sql.catalyst.expressions.Hypot | hypot | SELECT hypot(3, 4) 
| struct<HYPOT(CAST(3 AS DOUBLE), CAST(4 AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.If | if | SELECT if(1 < 2, 'a', 
'b') | struct<(IF((1 < 2), a, b)):string> |
+| org.apache.spark.sql.catalyst.expressions.IfNull | ifnull | SELECT 
ifnull(NULL, array('2')) | struct<ifnull(NULL, array('2')):array<string>> |
+| org.apache.spark.sql.catalyst.expressions.In | in | SELECT 1 in(1, 2, 3) | 
struct<(1 IN (1, 2, 3)):boolean> |
+| org.apache.spark.sql.catalyst.expressions.InitCap | initcap | SELECT 
initcap('sPark sql') | struct<initcap(sPark sql):string> |
+| org.apache.spark.sql.catalyst.expressions.Inline | inline | SELECT 
inline(array(struct(1, 'a'), struct(2, 'b'))) | struct<col1:int,col2:string> |
+| org.apache.spark.sql.catalyst.expressions.Inline | inline_outer | SELECT 
inline_outer(array(struct(1, 'a'), struct(2, 'b'))) | 
struct<col1:int,col2:string> |
+| org.apache.spark.sql.catalyst.expressions.InputFileBlockLength | 
input_file_block_length | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.InputFileBlockStart | 
input_file_block_start | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.InputFileName | input_file_name | 
N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.IntegralDivide | div | SELECT 3 
div 2 | struct<(3 div 2):bigint> |
+| org.apache.spark.sql.catalyst.expressions.IsNaN | isnan | SELECT 
isnan(cast('NaN' as double)) | struct<isnan(CAST(NaN AS DOUBLE)):boolean> |
+| org.apache.spark.sql.catalyst.expressions.IsNotNull | isnotnull | SELECT 
isnotnull(1) | struct<(1 IS NOT NULL):boolean> |
+| org.apache.spark.sql.catalyst.expressions.IsNull | isnull | SELECT isnull(1) 
| struct<(1 IS NULL):boolean> |
+| org.apache.spark.sql.catalyst.expressions.JsonObjectKeys | json_object_keys 
| SELECT json_object_keys('{}') | struct<json_object_keys({}):array<string>> |
+| org.apache.spark.sql.catalyst.expressions.JsonToStructs | from_json | SELECT 
from_json('{"a":1, "b":0.8}', 'a INT, b DOUBLE') | struct<from_json({"a":1, 
"b":0.8}):struct<a:int,b:double>> |
+| org.apache.spark.sql.catalyst.expressions.JsonTuple | json_tuple | SELECT 
json_tuple('{"a":1, "b":2}', 'a', 'b') | struct<c0:string,c1:string> |
+| org.apache.spark.sql.catalyst.expressions.Lag | lag | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.LastDay | last_day | SELECT 
last_day('2009-01-12') | struct<last_day(CAST(2009-01-12 AS DATE)):date> |
+| org.apache.spark.sql.catalyst.expressions.Lead | lead | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.Least | least | SELECT least(10, 
9, 2, 4, 3) | struct<least(10, 9, 2, 4, 3):int> |
+| org.apache.spark.sql.catalyst.expressions.Left | left | SELECT left('Spark 
SQL', 3) | struct<left('Spark SQL', 3):string> |
+| org.apache.spark.sql.catalyst.expressions.Length | character_length | SELECT 
character_length('Spark SQL ') | struct<character_length(Spark SQL ):int> |
+| org.apache.spark.sql.catalyst.expressions.Length | char_length | SELECT 
char_length('Spark SQL ') | struct<char_length(Spark SQL ):int> |
+| org.apache.spark.sql.catalyst.expressions.Length | length | SELECT 
length('Spark SQL ') | struct<length(Spark SQL ):int> |
+| org.apache.spark.sql.catalyst.expressions.LengthOfJsonArray | 
json_array_length | SELECT json_array_length('[1,2,3,4]') | 
struct<json_array_length([1,2,3,4]):int> |
+| org.apache.spark.sql.catalyst.expressions.LessThan | < | SELECT 1 < 2 | 
struct<(1 < 2):boolean> |
+| org.apache.spark.sql.catalyst.expressions.LessThanOrEqual | <= | SELECT 2 <= 
2 | struct<(2 <= 2):boolean> |
+| org.apache.spark.sql.catalyst.expressions.Levenshtein | levenshtein | SELECT 
levenshtein('kitten', 'sitting') | struct<levenshtein(kitten, sitting):int> |
+| org.apache.spark.sql.catalyst.expressions.Like | like | SELECT like('Spark', 
'_park') | struct<Spark LIKE _park:boolean> |
+| org.apache.spark.sql.catalyst.expressions.Log | ln | SELECT ln(1) | 
struct<LOG(CAST(1 AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.Log10 | log10 | SELECT log10(10) | 
struct<LOG10(CAST(10 AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.Log1p | log1p | SELECT log1p(0) | 
struct<LOG1P(CAST(0 AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.Log2 | log2 | SELECT log2(2) | 
struct<LOG2(CAST(2 AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.Logarithm | log | SELECT log(10, 
100) | struct<LOG(CAST(10 AS DOUBLE), CAST(100 AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.Lower | lcase | SELECT 
lcase('SparkSql') | struct<lower(SparkSql):string> |
+| org.apache.spark.sql.catalyst.expressions.Lower | lower | SELECT 
lower('SparkSql') | struct<lower(SparkSql):string> |
+| org.apache.spark.sql.catalyst.expressions.MakeDate | make_date | SELECT 
make_date(2013, 7, 15) | struct<make_date(2013, 7, 15):date> |
+| org.apache.spark.sql.catalyst.expressions.MakeInterval | make_interval | 
SELECT make_interval(100, 11, 1, 1, 12, 30, 01.001001) | 
struct<make_interval(100, 11, 1, 1, 12, 30, CAST(1.001001 AS 
DECIMAL(8,6))):interval> |
+| org.apache.spark.sql.catalyst.expressions.MakeTimestamp | make_timestamp | 
SELECT make_timestamp(2014, 12, 28, 6, 30, 45.887) | 
struct<make_timestamp(2014, 12, 28, 6, 30, CAST(45.887 AS 
DECIMAL(8,6))):timestamp> |
+| org.apache.spark.sql.catalyst.expressions.MapConcat | map_concat | SELECT 
map_concat(map(1, 'a', 2, 'b'), map(3, 'c')) | struct<map_concat(map(1, a, 2, 
b), map(3, c)):map<int,string>> |
+| org.apache.spark.sql.catalyst.expressions.MapEntries | map_entries | SELECT 
map_entries(map(1, 'a', 2, 'b')) | struct<map_entries(map(1, a, 2, 
b)):array<struct<key:int,value:string>>> |
+| org.apache.spark.sql.catalyst.expressions.MapFilter | map_filter | SELECT 
map_filter(map(1, 0, 2, 2, 3, -1), (k, v) -> k > v) | struct<map_filter(map(1, 
0, 2, 2, 3, -1), lambdafunction((namedlambdavariable() > 
namedlambdavariable()), namedlambdavariable(), 
namedlambdavariable())):map<int,int>> |
+| org.apache.spark.sql.catalyst.expressions.MapFromArrays | map_from_arrays | 
SELECT map_from_arrays(array(1.0, 3.0), array('2', '4')) | 
struct<map_from_arrays(array(1.0, 3.0), array(2, 4)):map<decimal(2,1),string>> |
+| org.apache.spark.sql.catalyst.expressions.MapFromEntries | map_from_entries 
| SELECT map_from_entries(array(struct(1, 'a'), struct(2, 'b'))) | 
struct<map_from_entries(array(named_struct(col1, 1, col2, a), 
named_struct(col1, 2, col2, b))):map<int,string>> |
+| org.apache.spark.sql.catalyst.expressions.MapKeys | map_keys | SELECT 
map_keys(map(1, 'a', 2, 'b')) | struct<map_keys(map(1, a, 2, b)):array<int>> |
+| org.apache.spark.sql.catalyst.expressions.MapValues | map_values | SELECT 
map_values(map(1, 'a', 2, 'b')) | struct<map_values(map(1, a, 2, 
b)):array<string>> |
+| org.apache.spark.sql.catalyst.expressions.MapZipWith | map_zip_with | SELECT 
map_zip_with(map(1, 'a', 2, 'b'), map(1, 'x', 2, 'y'), (k, v1, v2) -> 
concat(v1, v2)) | struct<map_zip_with(map(1, a, 2, b), map(1, x, 2, y), 
lambdafunction(concat(namedlambdavariable(), namedlambdavariable()), 
namedlambdavariable(), namedlambdavariable(), 
namedlambdavariable())):map<int,string>> |
+| org.apache.spark.sql.catalyst.expressions.Md5 | md5 | SELECT md5('Spark') | 
struct<md5(CAST(Spark AS BINARY)):string> |
+| org.apache.spark.sql.catalyst.expressions.Minute | minute | SELECT 
minute('2009-07-30 12:58:59') | struct<minute(CAST(2009-07-30 12:58:59 AS 
TIMESTAMP)):int> |
+| org.apache.spark.sql.catalyst.expressions.MonotonicallyIncreasingID | 
monotonically_increasing_id | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.Month | month | SELECT 
month('2016-07-30') | struct<month(CAST(2016-07-30 AS DATE)):int> |
+| org.apache.spark.sql.catalyst.expressions.MonthsBetween | months_between | 
SELECT months_between('1997-02-28 10:30:00', '1996-10-30') | 
struct<months_between(CAST(1997-02-28 10:30:00 AS TIMESTAMP), CAST(1996-10-30 
AS TIMESTAMP), true):double> |
+| org.apache.spark.sql.catalyst.expressions.Multiply | * | SELECT 2 * 3 | 
struct<(2 * 3):int> |
+| org.apache.spark.sql.catalyst.expressions.Murmur3Hash | hash | SELECT 
hash('Spark', array(123), 2) | struct<hash(Spark, array(123), 2):int> |
+| org.apache.spark.sql.catalyst.expressions.NTile | ntile | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.NaNvl | nanvl | SELECT 
nanvl(cast('NaN' as double), 123) | struct<nanvl(CAST(NaN AS DOUBLE), CAST(123 
AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.NamedStruct | struct | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.NextDay | next_day | SELECT 
next_day('2015-01-14', 'TU') | struct<next_day(CAST(2015-01-14 AS DATE), 
TU):date> |
+| org.apache.spark.sql.catalyst.expressions.Not | ! | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.Not | not | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.Now | now | SELECT now() | 
struct<now():timestamp> |
+| org.apache.spark.sql.catalyst.expressions.NullIf | nullif | SELECT nullif(2, 
2) | struct<nullif(2, 2):int> |
+| org.apache.spark.sql.catalyst.expressions.Nvl | nvl | SELECT nvl(NULL, 
array('2')) | struct<nvl(NULL, array('2')):array<string>> |
+| org.apache.spark.sql.catalyst.expressions.Nvl2 | nvl2 | SELECT nvl2(NULL, 2, 
1) | struct<nvl2(NULL, 2, 1):int> |
+| org.apache.spark.sql.catalyst.expressions.OctetLength | octet_length | 
SELECT octet_length('Spark SQL') | struct<octet_length(Spark SQL):int> |
+| org.apache.spark.sql.catalyst.expressions.Or | or | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.Overlay | overlay | SELECT 
overlay('Spark SQL' PLACING '_' FROM 6) | struct<overlay(Spark SQL, _, 6, 
-1):string> |
+| org.apache.spark.sql.catalyst.expressions.ParseToDate | to_date | SELECT 
to_date('2009-07-30 04:17:52') | struct<to_date('2009-07-30 04:17:52'):date> |
+| org.apache.spark.sql.catalyst.expressions.ParseToTimestamp | to_timestamp | 
SELECT to_timestamp('2016-12-31 00:12:00') | struct<to_timestamp('2016-12-31 
00:12:00'):timestamp> |
+| org.apache.spark.sql.catalyst.expressions.ParseUrl | parse_url | SELECT 
parse_url('http://spark.apache.org/path?query=1', 'HOST') | 
struct<parse_url(http://spark.apache.org/path?query=1, HOST):string> |
+| org.apache.spark.sql.catalyst.expressions.PercentRank | percent_rank | N/A | 
N/A |
+| org.apache.spark.sql.catalyst.expressions.Pi | pi | SELECT pi() | 
struct<PI():double> |
+| org.apache.spark.sql.catalyst.expressions.Pmod | pmod | SELECT pmod(10, 3) | 
struct<pmod(10, 3):int> |
+| org.apache.spark.sql.catalyst.expressions.PosExplode | posexplode_outer | 
SELECT posexplode_outer(array(10,20)) | struct<pos:int,col:int> |
+| org.apache.spark.sql.catalyst.expressions.PosExplode | posexplode | SELECT 
posexplode(array(10,20)) | struct<pos:int,col:int> |
+| org.apache.spark.sql.catalyst.expressions.Pow | pow | SELECT pow(2, 3) | 
struct<pow(CAST(2 AS DOUBLE), CAST(3 AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.Pow | power | SELECT power(2, 3) | 
struct<POWER(CAST(2 AS DOUBLE), CAST(3 AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.Quarter | quarter | SELECT 
quarter('2016-08-31') | struct<quarter(CAST(2016-08-31 AS DATE)):int> |
+| org.apache.spark.sql.catalyst.expressions.RLike | rlike | SELECT 
'%SystemDrive%\Users\John' rlike '%SystemDrive%\\Users.*' | 
struct<%SystemDrive%UsersJohn RLIKE %SystemDrive%\Users.*:boolean> |
+| org.apache.spark.sql.catalyst.expressions.Rand | random | SELECT random() | 
struct<rand():double> |
+| org.apache.spark.sql.catalyst.expressions.Rand | rand | SELECT rand() | 
struct<rand():double> |
+| org.apache.spark.sql.catalyst.expressions.Randn | randn | SELECT randn() | 
struct<randn():double> |
+| org.apache.spark.sql.catalyst.expressions.Rank | rank | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.RegExpExtract | regexp_extract | 
SELECT regexp_extract('100-200', '(\\d+)-(\\d+)', 1) | 
struct<regexp_extract(100-200, (\d+)-(\d+), 1):string> |
+| org.apache.spark.sql.catalyst.expressions.RegExpReplace | regexp_replace | 
SELECT regexp_replace('100-200', '(\\d+)', 'num') | 
struct<regexp_replace(100-200, (\d+), num):string> |
+| org.apache.spark.sql.catalyst.expressions.Remainder | % | SELECT 2 % 1.8 | 
struct<(CAST(CAST(2 AS DECIMAL(1,0)) AS DECIMAL(2,1)) % CAST(1.8 AS 
DECIMAL(2,1))):decimal(2,1)> |
+| org.apache.spark.sql.catalyst.expressions.Remainder | mod | SELECT 2 % 1.8 | 
struct<(CAST(CAST(2 AS DECIMAL(1,0)) AS DECIMAL(2,1)) % CAST(1.8 AS 
DECIMAL(2,1))):decimal(2,1)> |
+| org.apache.spark.sql.catalyst.expressions.Reverse | reverse | SELECT 
reverse('Spark SQL') | struct<reverse(Spark SQL):string> |
+| org.apache.spark.sql.catalyst.expressions.Right | right | SELECT 
right('Spark SQL', 3) | struct<right('Spark SQL', 3):string> |
+| org.apache.spark.sql.catalyst.expressions.Rint | rint | SELECT rint(12.3456) 
| struct<ROUND(CAST(12.3456 AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.Rollup | rollup | SELECT name, 
age, count(*) FROM VALUES (2, 'Alice'), (5, 'Bob') people(age, name) GROUP BY 
rollup(name, age) | struct<name:string,age:int,count(1):bigint> |
+| org.apache.spark.sql.catalyst.expressions.Round | round | SELECT round(2.5, 
0) | struct<round(2.5, 0):decimal(2,0)> |
+| org.apache.spark.sql.catalyst.expressions.RowNumber | row_number | N/A | N/A 
|
+| org.apache.spark.sql.catalyst.expressions.SchemaOfCsv | schema_of_csv | 
SELECT schema_of_csv('1,abc') | struct<schema_of_csv(1,abc):string> |
+| org.apache.spark.sql.catalyst.expressions.SchemaOfJson | schema_of_json | 
SELECT schema_of_json('[{"col":0}]') | 
struct<schema_of_json([{"col":0}]):string> |
+| org.apache.spark.sql.catalyst.expressions.Second | second | SELECT 
second('2009-07-30 12:58:59') | struct<second(CAST(2009-07-30 12:58:59 AS 
TIMESTAMP)):int> |
+| org.apache.spark.sql.catalyst.expressions.Sentences | sentences | SELECT 
sentences('Hi there! Good morning.') | struct<sentences(Hi there! Good 
morning., , ):array<array<string>>> |
+| org.apache.spark.sql.catalyst.expressions.Sequence | sequence | SELECT 
sequence(1, 5) | struct<sequence(1, 5):array<int>> |
+| org.apache.spark.sql.catalyst.expressions.Sha1 | sha1 | SELECT sha1('Spark') 
| struct<sha1(CAST(Spark AS BINARY)):string> |
+| org.apache.spark.sql.catalyst.expressions.Sha1 | sha | SELECT sha('Spark') | 
struct<sha(CAST(Spark AS BINARY)):string> |
+| org.apache.spark.sql.catalyst.expressions.Sha2 | sha2 | SELECT sha2('Spark', 
256) | struct<sha2(CAST(Spark AS BINARY), 256):string> |
+| org.apache.spark.sql.catalyst.expressions.ShiftLeft | shiftleft | SELECT 
shiftleft(2, 1) | struct<shiftleft(2, 1):int> |
+| org.apache.spark.sql.catalyst.expressions.ShiftRight | shiftright | SELECT 
shiftright(4, 1) | struct<shiftright(4, 1):int> |
+| org.apache.spark.sql.catalyst.expressions.ShiftRightUnsigned | 
shiftrightunsigned | SELECT shiftrightunsigned(4, 1) | 
struct<shiftrightunsigned(4, 1):int> |
+| org.apache.spark.sql.catalyst.expressions.Shuffle | shuffle | SELECT 
shuffle(array(1, 20, 3, 5)) | struct<shuffle(array(1, 20, 3, 5)):array<int>> |
+| org.apache.spark.sql.catalyst.expressions.Signum | signum | SELECT 
signum(40) | struct<SIGNUM(CAST(40 AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.Signum | sign | SELECT sign(40) | 
struct<sign(CAST(40 AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.Sin | sin | SELECT sin(0) | 
struct<SIN(CAST(0 AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.Sinh | sinh | SELECT sinh(0) | 
struct<SINH(CAST(0 AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.Size | size | SELECT 
size(array('b', 'd', 'c', 'a')) | struct<size(array(b, d, c, a)):int> |
+| org.apache.spark.sql.catalyst.expressions.Size | cardinality | SELECT 
cardinality(array('b', 'd', 'c', 'a')) | struct<cardinality(array(b, d, c, 
a)):int> |
+| org.apache.spark.sql.catalyst.expressions.Slice | slice | SELECT 
slice(array(1, 2, 3, 4), 2, 2) | struct<slice(array(1, 2, 3, 4), 2, 
2):array<int>> |
+| org.apache.spark.sql.catalyst.expressions.SortArray | sort_array | SELECT 
sort_array(array('b', 'd', null, 'c', 'a'), true) | struct<sort_array(array(b, 
d, CAST(NULL AS STRING), c, a), true):array<string>> |
+| org.apache.spark.sql.catalyst.expressions.SoundEx | soundex | SELECT 
soundex('Miller') | struct<soundex(Miller):string> |
+| org.apache.spark.sql.catalyst.expressions.SparkPartitionID | 
spark_partition_id | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.SparkVersion | version | N/A | N/A 
|
+| org.apache.spark.sql.catalyst.expressions.Sqrt | sqrt | SELECT sqrt(4) | 
struct<SQRT(CAST(4 AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.Stack | stack | SELECT stack(2, 1, 
2, 3) | struct<col0:int,col1:int> |
+| org.apache.spark.sql.catalyst.expressions.StringInstr | instr | SELECT 
instr('SparkSQL', 'SQL') | struct<instr(SparkSQL, SQL):int> |
+| org.apache.spark.sql.catalyst.expressions.StringLPad | lpad | SELECT 
lpad('hi', 5, '??') | struct<lpad(hi, 5, ??):string> |
+| org.apache.spark.sql.catalyst.expressions.StringLocate | position | SELECT 
position('bar', 'foobarbar') | struct<locate(bar, foobarbar, 1):int> |
+| org.apache.spark.sql.catalyst.expressions.StringLocate | locate | SELECT 
locate('bar', 'foobarbar') | struct<locate(bar, foobarbar, 1):int> |
+| org.apache.spark.sql.catalyst.expressions.StringRPad | rpad | SELECT 
rpad('hi', 5, '??') | struct<rpad(hi, 5, ??):string> |
+| org.apache.spark.sql.catalyst.expressions.StringRepeat | repeat | SELECT 
repeat('123', 2) | struct<repeat(123, 2):string> |
+| org.apache.spark.sql.catalyst.expressions.StringReplace | replace | SELECT 
replace('ABCabc', 'abc', 'DEF') | struct<replace(ABCabc, abc, DEF):string> |
+| org.apache.spark.sql.catalyst.expressions.StringSpace | space | SELECT 
concat(space(2), '1') | struct<concat(space(2), 1):string> |
+| org.apache.spark.sql.catalyst.expressions.StringSplit | split | SELECT 
split('oneAtwoBthreeC', '[ABC]') | struct<split(oneAtwoBthreeC, [ABC], 
-1):array<string>> |
+| org.apache.spark.sql.catalyst.expressions.StringToMap | str_to_map | SELECT 
str_to_map('a:1,b:2,c:3', ',', ':') | struct<str_to_map(a:1,b:2,c:3, ,, 
:):map<string,string>> |
+| org.apache.spark.sql.catalyst.expressions.StringTranslate | translate | 
SELECT translate('AaBbCc', 'abc', '123') | struct<translate(AaBbCc, abc, 
123):string> |
+| org.apache.spark.sql.catalyst.expressions.StringTrim | trim | SELECT trim('  
  SparkSQL   ') | struct<trim(    SparkSQL   ):string> |
+| org.apache.spark.sql.catalyst.expressions.StringTrimLeft | ltrim | SELECT 
ltrim('    SparkSQL   ') | struct<ltrim(    SparkSQL   ):string> |
+| org.apache.spark.sql.catalyst.expressions.StringTrimRight | rtrim | SELECT 
rtrim('    SparkSQL   ') | struct<rtrim(    SparkSQL   ):string> |
+| org.apache.spark.sql.catalyst.expressions.StructsToCsv | to_csv | SELECT 
to_csv(named_struct('a', 1, 'b', 2)) | struct<to_csv(named_struct(a, 1, b, 
2)):string> |
+| org.apache.spark.sql.catalyst.expressions.StructsToJson | to_json | SELECT 
to_json(named_struct('a', 1, 'b', 2)) | struct<to_json(named_struct(a, 1, b, 
2)):string> |
+| org.apache.spark.sql.catalyst.expressions.Substring | substr | SELECT 
substr('Spark SQL', 5) | struct<substr(Spark SQL, 5, 2147483647):string> |
+| org.apache.spark.sql.catalyst.expressions.Substring | substring | SELECT 
substring('Spark SQL', 5) | struct<substring(Spark SQL, 5, 2147483647):string> |
+| org.apache.spark.sql.catalyst.expressions.SubstringIndex | substring_index | 
SELECT substring_index('www.apache.org', '.', 2) | 
struct<substring_index(www.apache.org, ., 2):string> |
+| org.apache.spark.sql.catalyst.expressions.Subtract | - | SELECT 2 - 1 | 
struct<(2 - 1):int> |
+| org.apache.spark.sql.catalyst.expressions.Tan | tan | SELECT tan(0) | 
struct<TAN(CAST(0 AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.Tanh | tanh | SELECT tanh(0) | 
struct<TANH(CAST(0 AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.TimeWindow | window | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.ToDegrees | degrees | SELECT 
degrees(3.141592653589793) | struct<DEGREES(CAST(3.141592653589793 AS 
DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.ToRadians | radians | SELECT 
radians(180) | struct<RADIANS(CAST(180 AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.ToUTCTimestamp | to_utc_timestamp 
| SELECT to_utc_timestamp('2016-08-31', 'Asia/Seoul') | 
struct<to_utc_timestamp(CAST(2016-08-31 AS TIMESTAMP), Asia/Seoul):timestamp> |
+| org.apache.spark.sql.catalyst.expressions.ToUnixTimestamp | 
to_unix_timestamp | SELECT to_unix_timestamp('2016-04-08', 'yyyy-MM-dd') | 
struct<to_unix_timestamp(2016-04-08, yyyy-MM-dd):bigint> |
+| org.apache.spark.sql.catalyst.expressions.TransformKeys | transform_keys | 
SELECT transform_keys(map_from_arrays(array(1, 2, 3), array(1, 2, 3)), (k, v) 
-> k + 1) | struct<transform_keys(map_from_arrays(array(1, 2, 3), array(1, 2, 
3)), lambdafunction((namedlambdavariable() + 1), namedlambdavariable(), 
namedlambdavariable())):map<int,int>> |
+| org.apache.spark.sql.catalyst.expressions.TransformValues | transform_values 
| SELECT transform_values(map_from_arrays(array(1, 2, 3), array(1, 2, 3)), (k, 
v) -> v + 1) | struct<transform_values(map_from_arrays(array(1, 2, 3), array(1, 
2, 3)), lambdafunction((namedlambdavariable() + 1), namedlambdavariable(), 
namedlambdavariable())):map<int,int>> |
+| org.apache.spark.sql.catalyst.expressions.TruncDate | trunc | SELECT 
trunc('2019-08-04', 'week') | struct<trunc(CAST(2019-08-04 AS DATE), 
week):date> |
+| org.apache.spark.sql.catalyst.expressions.TruncTimestamp | date_trunc | 
SELECT date_trunc('YEAR', '2015-03-05T09:32:05.359') | struct<date_trunc(YEAR, 
CAST(2015-03-05T09:32:05.359 AS TIMESTAMP)):timestamp> |
+| org.apache.spark.sql.catalyst.expressions.TypeOf | typeof | SELECT typeof(1) 
| struct<typeof(1):string> |
+| org.apache.spark.sql.catalyst.expressions.UnBase64 | unbase64 | SELECT 
unbase64('U3BhcmsgU1FM') | struct<unbase64(U3BhcmsgU1FM):binary> |
+| org.apache.spark.sql.catalyst.expressions.UnaryMinus | negative | SELECT 
negative(1) | struct<(- 1):int> |
+| org.apache.spark.sql.catalyst.expressions.UnaryPositive | positive | N/A | 
N/A |
+| org.apache.spark.sql.catalyst.expressions.Unhex | unhex | SELECT 
decode(unhex('537061726B2053514C'), 'UTF-8') | 
struct<decode(unhex(537061726B2053514C), UTF-8):string> |
+| org.apache.spark.sql.catalyst.expressions.UnixTimestamp | unix_timestamp | 
SELECT unix_timestamp() | struct<unix_timestamp(current_timestamp(), yyyy-MM-dd 
HH:mm:ss):bigint> |
+| org.apache.spark.sql.catalyst.expressions.Upper | ucase | SELECT 
ucase('SparkSql') | struct<ucase(SparkSql):string> |
+| org.apache.spark.sql.catalyst.expressions.Upper | upper | SELECT 
upper('SparkSql') | struct<upper(SparkSql):string> |
+| org.apache.spark.sql.catalyst.expressions.Uuid | uuid | SELECT uuid() | 
struct<uuid():string> |
+| org.apache.spark.sql.catalyst.expressions.WeekDay | weekday | SELECT 
weekday('2009-07-30') | struct<weekday(CAST(2009-07-30 AS DATE)):int> |
+| org.apache.spark.sql.catalyst.expressions.WeekOfYear | weekofyear | SELECT 
weekofyear('2008-02-20') | struct<weekofyear(CAST(2008-02-20 AS DATE)):int> |
+| org.apache.spark.sql.catalyst.expressions.XxHash64 | xxhash64 | SELECT 
xxhash64('Spark', array(123), 2) | struct<xxhash64(Spark, array(123), 
2):bigint> |
+| org.apache.spark.sql.catalyst.expressions.Year | year | SELECT 
year('2016-07-30') | struct<year(CAST(2016-07-30 AS DATE)):int> |
+| org.apache.spark.sql.catalyst.expressions.ZipWith | zip_with | SELECT 
zip_with(array(1, 2, 3), array('a', 'b', 'c'), (x, y) -> (y, x)) | 
struct<zip_with(array(1, 2, 3), array(a, b, c), lambdafunction(named_struct(y, 
namedlambdavariable(), x, namedlambdavariable()), namedlambdavariable(), 
namedlambdavariable())):array<struct<y:string,x:int>>> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile | 
approx_percentile | SELECT approx_percentile(10.0, array(0.5, 0.4, 0.1), 100) | 
struct<approx_percentile(10.0, array(0.5, 0.4, 0.1), 100):array<decimal(3,1)>> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile | 
percentile_approx | SELECT percentile_approx(10.0, array(0.5, 0.4, 0.1), 100) | 
struct<percentile_approx(10.0, array(0.5, 0.4, 0.1), 100):array<decimal(3,1)>> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.Average | avg | SELECT 
avg(col) FROM VALUES (1), (2), (3) AS tab(col) | struct<avg(col):double> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.Average | mean | SELECT 
mean(col) FROM VALUES (1), (2), (3) AS tab(col) | struct<mean(col):double> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.BitAndAgg | bit_and | 
SELECT bit_and(col) FROM VALUES (3), (5) AS tab(col) | struct<bit_and(col):int> 
|
+| org.apache.spark.sql.catalyst.expressions.aggregate.BitOrAgg | bit_or | 
SELECT bit_or(col) FROM VALUES (3), (5) AS tab(col) | struct<bit_or(col):int> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.BitXorAgg | bit_xor | 
SELECT bit_xor(col) FROM VALUES (3), (5) AS tab(col) | struct<bit_xor(col):int> 
|
+| org.apache.spark.sql.catalyst.expressions.aggregate.BoolAnd | every | SELECT 
every(col) FROM VALUES (true), (true), (true) AS tab(col) | 
struct<every(col):boolean> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.BoolAnd | bool_and | 
SELECT bool_and(col) FROM VALUES (true), (true), (true) AS tab(col) | 
struct<bool_and(col):boolean> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.BoolOr | bool_or | 
SELECT bool_or(col) FROM VALUES (true), (false), (false) AS tab(col) | 
struct<bool_or(col):boolean> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.BoolOr | some | SELECT 
some(col) FROM VALUES (true), (false), (false) AS tab(col) | 
struct<some(col):boolean> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.BoolOr | any | SELECT 
any(col) FROM VALUES (true), (false), (false) AS tab(col) | 
struct<any(col):boolean> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.CollectList | 
collect_list | SELECT collect_list(col) FROM VALUES (1), (2), (1) AS tab(col) | 
struct<collect_list(col):array<int>> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.CollectSet | collect_set 
| SELECT collect_set(col) FROM VALUES (1), (2), (1) AS tab(col) | 
struct<collect_set(col):array<int>> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.Corr | corr | SELECT 
corr(c1, c2) FROM VALUES (3, 2), (3, 3), (6, 4) as tab(c1, c2) | 
struct<corr(CAST(c1 AS DOUBLE), CAST(c2 AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.Count | count | SELECT 
count(*) FROM VALUES (NULL), (5), (5), (20) AS tab(col) | 
struct<count(1):bigint> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.CountIf | count_if | 
SELECT count_if(col % 2 = 0) FROM VALUES (NULL), (0), (1), (2), (3) AS tab(col) 
| struct<count_if(((col % 2) = 0)):bigint> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.CountMinSketchAgg | 
count_min_sketch | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.aggregate.CovPopulation | 
covar_pop | SELECT covar_pop(c1, c2) FROM VALUES (1,1), (2,2), (3,3) AS tab(c1, 
c2) | struct<covar_pop(CAST(c1 AS DOUBLE), CAST(c2 AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.CovSample | covar_samp | 
SELECT covar_samp(c1, c2) FROM VALUES (1,1), (2,2), (3,3) AS tab(c1, c2) | 
struct<covar_samp(CAST(c1 AS DOUBLE), CAST(c2 AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.First | first_value | 
SELECT first_value(col) FROM VALUES (10), (5), (20) AS tab(col) | 
struct<first_value(col, false):int> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.First | first | SELECT 
first(col) FROM VALUES (10), (5), (20) AS tab(col) | struct<first(col, 
false):int> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.HyperLogLogPlusPlus | 
approx_count_distinct | SELECT approx_count_distinct(col1) FROM VALUES (1), 
(1), (2), (2), (3) tab(col1) | struct<approx_count_distinct(col1):bigint> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.Kurtosis | kurtosis | 
SELECT kurtosis(col) FROM VALUES (-10), (-20), (100), (1000) AS tab(col) | 
struct<kurtosis(CAST(col AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.Last | last_value | 
SELECT last_value(col) FROM VALUES (10), (5), (20) AS tab(col) | 
struct<last_value(col, false):int> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.Last | last | SELECT 
last(col) FROM VALUES (10), (5), (20) AS tab(col) | struct<last(col, 
false):int> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.Max | max | SELECT 
max(col) FROM VALUES (10), (50), (20) AS tab(col) | struct<max(col):int> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.MaxBy | max_by | SELECT 
max_by(x, y) FROM VALUES (('a', 10)), (('b', 50)), (('c', 20)) AS tab(x, y) | 
struct<maxby(x, y):string> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.Min | min | SELECT 
min(col) FROM VALUES (10), (-1), (20) AS tab(col) | struct<min(col):int> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.MinBy | min_by | SELECT 
min_by(x, y) FROM VALUES (('a', 10)), (('b', 50)), (('c', 20)) AS tab(x, y) | 
struct<minby(x, y):string> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.Percentile | percentile 
| SELECT percentile(col, 0.3) FROM VALUES (0), (10) AS tab(col) | 
struct<percentile(col, CAST(0.3 AS DOUBLE), 1):double> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.Skewness | skewness | 
SELECT skewness(col) FROM VALUES (-10), (-20), (100), (1000) AS tab(col) | 
struct<skewness(CAST(col AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.StddevPop | stddev_pop | 
SELECT stddev_pop(col) FROM VALUES (1), (2), (3) AS tab(col) | 
struct<stddev_pop(CAST(col AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.StddevSamp | stddev_samp 
| SELECT stddev_samp(col) FROM VALUES (1), (2), (3) AS tab(col) | 
struct<stddev_samp(CAST(col AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.StddevSamp | stddev | 
SELECT stddev(col) FROM VALUES (1), (2), (3) AS tab(col) | 
struct<stddev(CAST(col AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.StddevSamp | std | 
SELECT std(col) FROM VALUES (1), (2), (3) AS tab(col) | struct<std(CAST(col AS 
DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.Sum | sum | SELECT 
sum(col) FROM VALUES (5), (10), (15) AS tab(col) | struct<sum(col):bigint> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.VariancePop | var_pop | 
SELECT var_pop(col) FROM VALUES (1), (2), (3) AS tab(col) | 
struct<var_pop(CAST(col AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.VarianceSamp | var_samp 
| SELECT var_samp(col) FROM VALUES (1), (2), (3) AS tab(col) | 
struct<var_samp(CAST(col AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.VarianceSamp | variance 
| SELECT variance(col) FROM VALUES (1), (2), (3) AS tab(col) | 
struct<variance(CAST(col AS DOUBLE)):double> |
+| org.apache.spark.sql.catalyst.expressions.xml.XPathBoolean | xpath_boolean | 
SELECT xpath_boolean('<a><b>1</b></a>','a/b') | 
struct<xpath_boolean(<a><b>1</b></a>, a/b):boolean> |
+| org.apache.spark.sql.catalyst.expressions.xml.XPathDouble | xpath_number | 
SELECT xpath_number('<a><b>1</b><b>2</b></a>', 'sum(a/b)') | 
struct<xpath_number(<a><b>1</b><b>2</b></a>, sum(a/b)):double> |
+| org.apache.spark.sql.catalyst.expressions.xml.XPathDouble | xpath_double | 
SELECT xpath_double('<a><b>1</b><b>2</b></a>', 'sum(a/b)') | 
struct<xpath_double(<a><b>1</b><b>2</b></a>, sum(a/b)):double> |
+| org.apache.spark.sql.catalyst.expressions.xml.XPathFloat | xpath_float | 
SELECT xpath_float('<a><b>1</b><b>2</b></a>', 'sum(a/b)') | 
struct<xpath_float(<a><b>1</b><b>2</b></a>, sum(a/b)):float> |
+| org.apache.spark.sql.catalyst.expressions.xml.XPathInt | xpath_int | SELECT 
xpath_int('<a><b>1</b><b>2</b></a>', 'sum(a/b)') | 
struct<xpath_int(<a><b>1</b><b>2</b></a>, sum(a/b)):int> |
+| org.apache.spark.sql.catalyst.expressions.xml.XPathList | xpath | SELECT 
xpath('<a><b>b1</b><b>b2</b><b>b3</b><c>c1</c><c>c2</c></a>','a/b/text()') | 
struct<xpath(<a><b>b1</b><b>b2</b><b>b3</b><c>c1</c><c>c2</c></a>, 
a/b/text()):array<string>> |
+| org.apache.spark.sql.catalyst.expressions.xml.XPathLong | xpath_long | 
SELECT xpath_long('<a><b>1</b><b>2</b></a>', 'sum(a/b)') | 
struct<xpath_long(<a><b>1</b><b>2</b></a>, sum(a/b)):bigint> |
+| org.apache.spark.sql.catalyst.expressions.xml.XPathShort | xpath_short | 
SELECT xpath_short('<a><b>1</b><b>2</b></a>', 'sum(a/b)') | 
struct<xpath_short(<a><b>1</b><b>2</b></a>, sum(a/b)):smallint> |
+| org.apache.spark.sql.catalyst.expressions.xml.XPathString | xpath_string | 
SELECT xpath_string('<a><b>b</b><c>cc</c></a>','a/c') | 
struct<xpath_string(<a><b>b</b><c>cc</c></a>, a/c):string> |
\ No newline at end of file
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/ExpressionsSchemaSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/ExpressionsSchemaSuite.scala
new file mode 100644
index 0000000..dd72473
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/ExpressionsSchemaSuite.scala
@@ -0,0 +1,187 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import java.io.File
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.util.{fileToString, stringToFile}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.tags.ExtendedSQLTest
+
+// scalastyle:off line.size.limit
+/**
+ * End-to-end test cases for SQL schemas of expression examples.
+ * The golden result file is 
"spark/sql/core/src/test/resources/sql-functions/sql-expression-schema.md".
+ *
+ * To run the entire test suite:
+ * {{{
+ *   build/sbt "sql/test-only *ExpressionsSchemaSuite"
+ * }}}
+ *
+ * To re-generate golden files for entire suite, run:
+ * {{{
+ *   SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/test-only 
*ExpressionsSchemaSuite"
+ * }}}
+ *
+ * For example:
+ * {{{
+ *   ...
+ *   @ExpressionDescription(
+ *     usage = "_FUNC_(str, n) - Returns the string which repeats the given 
string value n times.",
+ *     examples = """
+ *       Examples:
+ *         > SELECT _FUNC_('123', 2);
+ *          123123
+ *     """,
+ *     since = "1.5.0")
+ *   case class StringRepeat(str: Expression, times: Expression)
+ *   ...
+ * }}}
+ *
+ * The format for golden result files look roughly like:
+ * {{{
+ *   ...
+ *   | org.apache.spark.sql.catalyst.expressions.StringRepeat | repeat | 
SELECT repeat('123', 2) | struct<repeat(123, 2):string> |
+ *   ...
+ * }}}
+ */
+// scalastyle:on line.size.limit
+@ExtendedSQLTest
+class ExpressionsSchemaSuite extends QueryTest with SharedSparkSession {
+
+  private val regenerateGoldenFiles: Boolean = 
System.getenv("SPARK_GENERATE_GOLDEN_FILES") == "1"
+
+  private val baseResourcePath = {
+    // We use a path based on Spark home for 2 reasons:
+    //   1. Maven can't get correct resource directory when resources in other 
jars.
+    //   2. We test subclasses in the hive-thriftserver module.
+    val sparkHome = {
+      assert(sys.props.contains("spark.test.home") ||
+        sys.env.contains("SPARK_HOME"), "spark.test.home or SPARK_HOME is not 
set.")
+      sys.props.getOrElse("spark.test.home", sys.env("SPARK_HOME"))
+    }
+
+    java.nio.file.Paths.get(sparkHome,
+      "sql", "core", "src", "test", "resources", "sql-functions").toFile
+  }
+
+  private val resultFile = new File(baseResourcePath, 
"sql-expression-schema.md")
+
+  /** A single SQL query's SQL and schema. */
+  protected case class QueryOutput(
+      className: String,
+      funcName: String,
+      sql: String = "N/A",
+      schema: String = "N/A") {
+    override def toString: String = {
+      s"| $className | $funcName | $sql | $schema |"
+    }
+  }
+
+  test("Check schemas for expression examples") {
+    val exampleRe = """^(.+);\n(?s)(.+)$""".r
+    val funInfos = spark.sessionState.functionRegistry.listFunction().map { 
funcId =>
+      spark.sessionState.catalog.lookupFunctionInfo(funcId)
+    }
+
+    val classFunsMap = funInfos.groupBy(_.getClassName).toSeq.sortBy(_._1)
+    val outputBuffer = new ArrayBuffer[String]
+    val outputs = new ArrayBuffer[QueryOutput]
+    val missingExamples = new ArrayBuffer[String]
+
+    classFunsMap.foreach { kv =>
+      val className = kv._1
+      kv._2.foreach { funInfo =>
+        val example = funInfo.getExamples
+        val funcName = funInfo.getName.replaceAll("\\|", "&#124;")
+        if (example == "") {
+          val queryOutput = QueryOutput(className, funcName)
+          outputBuffer += queryOutput.toString
+          outputs += queryOutput
+          missingExamples += funcName
+        }
+
+        // If expression exists 'Examples' segment, the first element is 
'Examples'. Because
+        // this test case is only used to print aliases of expressions for 
double checking.
+        // Therefore, we only need to output the first SQL and its 
corresponding schema.
+        // Note: We need to filter out the commands that set the parameters, 
such as:
+        // SET spark.sql.parser.escapedStringLiterals=true
+        example.split("  > 
").tail.filterNot(_.trim.startsWith("SET")).take(1).foreach {
+          case exampleRe(sql, _) =>
+            val df = spark.sql(sql)
+            val escapedSql = sql.replaceAll("\\|", "&#124;")
+            val schema = df.schema.catalogString.replaceAll("\\|", "&#124;")
+            val queryOutput = QueryOutput(className, funcName, escapedSql, 
schema)
+            outputBuffer += queryOutput.toString
+            outputs += queryOutput
+          case _ =>
+        }
+      }
+    }
+
+    if (regenerateGoldenFiles) {
+      val missingExampleStr = missingExamples.mkString(",")
+      val goldenOutput = {
+        s"<!-- Automatically generated by${getClass.getSimpleName} -->\n" +
+        "## Summary\n" +
+        s"  - Number of queries: ${outputs.size}\n" +
+        s"  - Number of expressions that missing example: 
${missingExamples.size}\n" +
+        s"  - Expressions missing examples: $missingExampleStr\n" +
+        "## Schema of Built-in Functions\n" +
+        "| Class name | Function name or alias | Query example | Output schema 
|\n" +
+        "| ---------- | ---------------------- | ------------- | ------------- 
|\n" +
+        outputBuffer.mkString("\n")
+      }
+      val parent = resultFile.getParentFile
+      if (!parent.exists()) {
+        assert(parent.mkdirs(), "Could not create directory: " + parent)
+      }
+      stringToFile(resultFile, goldenOutput)
+    }
+
+    val expectedOutputs: Seq[QueryOutput] = {
+      val goldenOutput = fileToString(resultFile)
+      val lines = goldenOutput.split("\n")
+
+      // The header of golden file has one line, plus four lines of the 
summary and three
+      // lines of the header of schema table.
+      assert(lines.size == outputs.size + 8,
+        s"Expected ${outputs.size + 8} blocks in result file but got 
${lines.size}. " +
+          s"Try regenerate the result files.")
+
+      Seq.tabulate(outputs.size) { i =>
+        val segments = lines(i + 8).split('|')
+        QueryOutput(
+          className = segments(1).trim,
+          funcName = segments(2).trim,
+          sql = segments(3).trim,
+          schema = segments(4).trim)
+      }
+    }
+
+    // Compare results.
+    assert(expectedOutputs.size == outputs.size, s"Number of queries not 
equals")
+
+    outputs.zip(expectedOutputs).foreach { case (output, expected) =>
+      assert(expected.sql == output.sql, "SQL query did not match")
+      assert(expected.schema == output.schema, s"Schema did not match for 
query ${expected.sql}")
+    }
+  }
+}


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

Reply via email to