From 08f0906dc0e0843abe60ace5e31fe7a64ab897bc Mon Sep 17 00:00:00 2001 From: gengjiaan Date: Thu, 9 Apr 2020 15:01:32 +0800 Subject: [PATCH 1/5] Support pretty name --- .../apache/spark/sql/catalyst/expressions/TimeWindow.scala | 1 + .../sql/catalyst/expressions/aggregate/MaxByAndMinBy.scala | 6 ++++++ .../apache/spark/sql/catalyst/expressions/arithmetic.scala | 4 ++-- .../spark/sql/catalyst/expressions/bitwiseExpressions.scala | 4 ++-- 4 files changed, 11 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala index caacb71814f1..82d689477080 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala @@ -63,6 +63,7 @@ case class TimeWindow( override def dataType: DataType = new StructType() .add(StructField("start", TimestampType)) .add(StructField("end", TimestampType)) + override def prettyName: String = "window" // This expression is replaced in the analyzer. override lazy val resolved = false 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 b69b341b0ee3..536100973e03 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 @@ -100,6 +100,9 @@ abstract class MaxMinBy extends DeclarativeAggregate { """, since = "3.0.0") case class MaxBy(valueExpr: Expression, orderingExpr: Expression) extends MaxMinBy { + + override def prettyName: String = "max_by" + override protected def funcName: String = "max_by" override protected def predicate(oldExpr: Expression, newExpr: Expression): Expression = @@ -118,6 +121,9 @@ case class MaxBy(valueExpr: Expression, orderingExpr: Expression) extends MaxMin """, since = "3.0.0") case class MinBy(valueExpr: Expression, orderingExpr: Expression) extends MaxMinBy { + + override def prettyName: String = "min_by" + override protected def funcName: String = "min_by" override protected def predicate(oldExpr: Expression, newExpr: Expression): Expression = diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index 6a64819aabb4..50658491a3d4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -43,6 +43,8 @@ case class UnaryMinus(child: Expression) extends UnaryExpression override def toString: String = s"-$child" + override def prettyName: String = "-" + private lazy val numeric = TypeUtils.getNumeric(dataType, checkOverflow) override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = dataType match { @@ -85,8 +87,6 @@ case class UnaryMinus(child: Expression) extends UnaryExpression case CalendarIntervalType => IntervalUtils.negate(input.asInstanceOf[CalendarInterval]) case _ => numeric.negate(input) } - - override def sql: String = s"(- ${child.sql})" } @ExpressionDescription( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwiseExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwiseExpressions.scala index 72a8f7e99729..7b819db32e42 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwiseExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwiseExpressions.scala @@ -172,6 +172,8 @@ case class BitwiseCount(child: Expression) extends UnaryExpression with ExpectsI override def toString: String = s"bit_count($child)" + override def prettyName: String = "bit_count" + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = child.dataType match { case BooleanType => defineCodeGen(ctx, ev, c => s"if ($c) 1 else 0") case _ => defineCodeGen(ctx, ev, c => s"java.lang.Long.bitCount($c)") @@ -184,6 +186,4 @@ case class BitwiseCount(child: Expression) extends UnaryExpression with ExpectsI case IntegerType => java.lang.Long.bitCount(input.asInstanceOf[Int]) case LongType => java.lang.Long.bitCount(input.asInstanceOf[Long]) } - - override def sql: String = s"bit_count(${child.sql})" } From 7b7f8bf5d9b2f5d929c8c9e26f76e61fe7423d6a Mon Sep 17 00:00:00 2001 From: gengjiaan Date: Thu, 9 Apr 2020 15:59:28 +0800 Subject: [PATCH 2/5] Adjust UT --- .../sql/catalyst/expressions/ExpressionSQLBuilderSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSQLBuilderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSQLBuilderSuite.scala index 492d97ba9d52..b9dbf5d39e5e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSQLBuilderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSQLBuilderSuite.scala @@ -131,8 +131,8 @@ class ExpressionSQLBuilderSuite extends SparkFunSuite { checkSQL('a.int / 'b.int, "(`a` / `b`)") checkSQL('a.int % 'b.int, "(`a` % `b`)") - checkSQL(-'a.int, "(- `a`)") - checkSQL(-('a.int + 'b.int), "(- (`a` + `b`))") + checkSQL(-'a.int, "-(`a`)") + checkSQL(-('a.int + 'b.int), "-((`a` + `b`))") } test("window specification") { From 67c868b64bc5727a640b27c266384598bc9f1e61 Mon Sep 17 00:00:00 2001 From: beliefer Date: Thu, 9 Apr 2020 22:42:29 +0800 Subject: [PATCH 3/5] regenerate golden files. --- .../sql-tests/results/ansi/interval.sql.out | 12 +++++----- .../sql-tests/results/ansi/literals.sql.out | 6 ++--- .../sql-tests/results/interval.sql.out | 14 +++++------ .../sql-tests/results/literals.sql.out | 6 ++--- .../sql-tests/results/operators.sql.out | 22 ++++++++--------- .../results/postgreSQL/window_part2.sql.out | 24 +++++++++---------- .../results/postgreSQL/window_part3.sql.out | 4 ++-- .../native/promoteStrings.sql.out | 2 +- 8 files changed, 45 insertions(+), 45 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out index d4238c73e39e..51c0e0aa758f 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out @@ -98,7 +98,7 @@ NULL -- !query select -interval '-1 month 1 day -1 second' -- !query schema -struct<(- INTERVAL '-1 months 1 days -1 seconds'):interval> +struct<-(INTERVAL '-1 months 1 days -1 seconds'):interval> -- !query output 1 months -1 days 1 seconds @@ -106,7 +106,7 @@ struct<(- INTERVAL '-1 months 1 days -1 seconds'):interval> -- !query select -interval -1 month 1 day -1 second -- !query schema -struct<(- INTERVAL '-1 months 1 days -1 seconds'):interval> +struct<-(INTERVAL '-1 months 1 days -1 seconds'):interval> -- !query output 1 months -1 days 1 seconds @@ -689,7 +689,7 @@ select interval '2-2' year to month + dateval from interval_arithmetic -- !query schema -struct +struct -- !query output 2012-01-01 2009-11-01 2014-03-01 2014-03-01 2009-11-01 2009-11-01 2014-03-01 @@ -705,7 +705,7 @@ select interval '2-2' year to month + tsval from interval_arithmetic -- !query schema -struct +struct -- !query output 2012-01-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 2014-03-01 00:00:00 2009-11-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 @@ -732,7 +732,7 @@ select interval '99 11:22:33.123456789' day to second + dateval from interval_arithmetic -- !query schema -struct +struct -- !query output 2012-01-01 2011-09-23 2012-04-09 2012-04-09 2011-09-23 2011-09-23 2012-04-09 @@ -748,7 +748,7 @@ select interval '99 11:22:33.123456789' day to second + tsval from interval_arithmetic -- !query schema -struct +struct -- !query output 2012-01-01 00:00:00 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 2012-04-09 11:22:33.123456 2011-09-23 12:37:26.876544 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out index f6720f6c5faa..1bd0c7646c4e 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out @@ -460,7 +460,7 @@ select -date '1999-01-01' struct<> -- !query output org.apache.spark.sql.AnalysisException -cannot resolve '(- DATE '1999-01-01')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'DATE '1999-01-01'' is of date type.; line 1 pos 7 +cannot resolve '-(DATE '1999-01-01')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'DATE '1999-01-01'' is of date type.; line 1 pos 7 -- !query @@ -469,7 +469,7 @@ select -timestamp '1999-01-01' struct<> -- !query output org.apache.spark.sql.AnalysisException -cannot resolve '(- TIMESTAMP '1999-01-01 00:00:00')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'TIMESTAMP '1999-01-01 00:00:00'' is of timestamp type.; line 1 pos 7 +cannot resolve '-(TIMESTAMP '1999-01-01 00:00:00')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'TIMESTAMP '1999-01-01 00:00:00'' is of timestamp type.; line 1 pos 7 -- !query @@ -478,4 +478,4 @@ select -x'2379ACFe' struct<> -- !query output org.apache.spark.sql.AnalysisException -cannot resolve '(- X'2379ACFE')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'X'2379ACFE'' is of binary type.; line 1 pos 7 +cannot resolve '-(X'2379ACFE')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'X'2379ACFE'' is of binary type.; line 1 pos 7 diff --git a/sql/core/src/test/resources/sql-tests/results/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/interval.sql.out index 0d758ca1f24c..0f75c2534df5 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval.sql.out @@ -93,7 +93,7 @@ NULL -- !query select -interval '-1 month 1 day -1 second' -- !query schema -struct<(- INTERVAL '-1 months 1 days -1 seconds'):interval> +struct<-(INTERVAL '-1 months 1 days -1 seconds'):interval> -- !query output 1 months -1 days 1 seconds @@ -101,7 +101,7 @@ struct<(- INTERVAL '-1 months 1 days -1 seconds'):interval> -- !query select -interval -1 month 1 day -1 second -- !query schema -struct<(- INTERVAL '-1 months 1 days -1 seconds'):interval> +struct<-(INTERVAL '-1 months 1 days -1 seconds'):interval> -- !query output 1 months -1 days 1 seconds @@ -668,7 +668,7 @@ select interval '2-2' year to month + dateval from interval_arithmetic -- !query schema -struct +struct -- !query output 2012-01-01 2009-11-01 2014-03-01 2014-03-01 2009-11-01 2009-11-01 2014-03-01 @@ -684,7 +684,7 @@ select interval '2-2' year to month + tsval from interval_arithmetic -- !query schema -struct +struct -- !query output 2012-01-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 2014-03-01 00:00:00 2009-11-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 @@ -711,7 +711,7 @@ select interval '99 11:22:33.123456789' day to second + dateval from interval_arithmetic -- !query schema -struct +struct -- !query output 2012-01-01 2011-09-23 2012-04-09 2012-04-09 2011-09-23 2011-09-23 2012-04-09 @@ -727,7 +727,7 @@ select interval '99 11:22:33.123456789' day to second + tsval from interval_arithmetic -- !query schema -struct +struct -- !query output 2012-01-01 00:00:00 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 2012-04-09 11:22:33.123456 2011-09-23 12:37:26.876544 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 @@ -831,7 +831,7 @@ select interval '\n-\t10\t 12:34:46.789\t' day to second -- !query select -(a) from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) -- !query schema -struct<(- a):interval> +struct<-(a):interval> -- !query output -178956970 years -8 months diff --git a/sql/core/src/test/resources/sql-tests/results/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/literals.sql.out index f6720f6c5faa..1bd0c7646c4e 100644 --- a/sql/core/src/test/resources/sql-tests/results/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/literals.sql.out @@ -460,7 +460,7 @@ select -date '1999-01-01' struct<> -- !query output org.apache.spark.sql.AnalysisException -cannot resolve '(- DATE '1999-01-01')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'DATE '1999-01-01'' is of date type.; line 1 pos 7 +cannot resolve '-(DATE '1999-01-01')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'DATE '1999-01-01'' is of date type.; line 1 pos 7 -- !query @@ -469,7 +469,7 @@ select -timestamp '1999-01-01' struct<> -- !query output org.apache.spark.sql.AnalysisException -cannot resolve '(- TIMESTAMP '1999-01-01 00:00:00')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'TIMESTAMP '1999-01-01 00:00:00'' is of timestamp type.; line 1 pos 7 +cannot resolve '-(TIMESTAMP '1999-01-01 00:00:00')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'TIMESTAMP '1999-01-01 00:00:00'' is of timestamp type.; line 1 pos 7 -- !query @@ -478,4 +478,4 @@ select -x'2379ACFe' struct<> -- !query output org.apache.spark.sql.AnalysisException -cannot resolve '(- X'2379ACFE')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'X'2379ACFE'' is of binary type.; line 1 pos 7 +cannot resolve '-(X'2379ACFE')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'X'2379ACFE'' is of binary type.; line 1 pos 7 diff --git a/sql/core/src/test/resources/sql-tests/results/operators.sql.out b/sql/core/src/test/resources/sql-tests/results/operators.sql.out index cf857cf9f98a..de482a7c9319 100644 --- a/sql/core/src/test/resources/sql-tests/results/operators.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/operators.sql.out @@ -37,7 +37,7 @@ struct<(+ 6.8):double> -- !query select -key, +key from testdata where key = 2 -- !query schema -struct<(- key):int,(+ key):int> +struct<-(key):int,(+ key):int> -- !query output -2 2 @@ -45,7 +45,7 @@ struct<(- key):int,(+ key):int> -- !query select -(key + 1), - key + 1, +(key + 5) from testdata where key = 1 -- !query schema -struct<(- (key + 1)):int,((- key) + 1):int,(+ (key + 5)):int> +struct<-((key + 1)):int,(-(key) + 1):int,(+ (key + 5)):int> -- !query output -2 0 6 @@ -53,7 +53,7 @@ struct<(- (key + 1)):int,((- key) + 1):int,(+ (key + 5)):int> -- !query select -max(key), +max(key) from testdata -- !query schema -struct<(- max(key)):int,(+ max(key)):int> +struct<-(max(key)):int,(+ max(key)):int> -- !query output -100 100 @@ -61,7 +61,7 @@ struct<(- max(key)):int,(+ max(key)):int> -- !query select - (-10) -- !query schema -struct<(- -10):int> +struct<-(-10):int> -- !query output 10 @@ -69,7 +69,7 @@ struct<(- -10):int> -- !query select + (-key) from testdata where key = 32 -- !query schema -struct<(+ (- key)):int> +struct<(+ -(key)):int> -- !query output -32 @@ -77,7 +77,7 @@ struct<(+ (- key)):int> -- !query select - (+max(key)) from testdata -- !query schema -struct<(- (+ max(key))):int> +struct<-((+ max(key))):int> -- !query output -100 @@ -85,7 +85,7 @@ struct<(- (+ max(key))):int> -- !query select - - 3 -- !query schema -struct<(- -3):int> +struct<-(-3):int> -- !query output 3 @@ -93,7 +93,7 @@ struct<(- -3):int> -- !query select - + 20 -- !query schema -struct<(- (+ 20)):int> +struct<-((+ 20)):int> -- !query output -20 @@ -109,7 +109,7 @@ struct<(+ (+ 100)):int> -- !query select - - max(key) from testdata -- !query schema -struct<(- (- max(key))):int> +struct<-(-(max(key))):int> -- !query output 100 @@ -117,7 +117,7 @@ struct<(- (- max(key))):int> -- !query select + - key from testdata where key = 33 -- !query schema -struct<(+ (- key)):int> +struct<(+ -(key)):int> -- !query output -33 @@ -437,7 +437,7 @@ struct -- !query select positive('-1.11'), positive(-1.11), negative('-1.11'), negative(-1.11) -- !query schema -struct<(+ CAST(-1.11 AS DOUBLE)):double,(+ -1.11):decimal(3,2),(- CAST(-1.11 AS DOUBLE)):double,(- -1.11):decimal(3,2)> +struct<(+ CAST(-1.11 AS DOUBLE)):double,(+ -1.11):decimal(3,2),-(CAST(-1.11 AS DOUBLE)):double,-(-1.11):decimal(3,2)> -- !query output -1.11 -1.11 1.11 1.11 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out index f41659a196ae..f3bef084d096 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out @@ -101,7 +101,7 @@ from window w as (order by ss.id asc nulls first range between 2 preceding and 2 following) -- !query schema -struct +struct -- !query output 1 1 1 3 2 2 1 4 @@ -123,7 +123,7 @@ from window w as (order by ss.id asc nulls last range between 2 preceding and 2 following) -- !query schema -struct +struct -- !query output 1 1 1 3 2 2 1 4 @@ -145,7 +145,7 @@ from window w as (order by ss.id desc nulls first range between 2 preceding and 2 following) -- !query schema -struct +struct -- !query output 1 1 3 1 2 2 4 1 @@ -167,7 +167,7 @@ from window w as (order by ss.id desc nulls last range between 2 preceding and 2 following) -- !query schema -struct +struct -- !query output 1 1 3 1 2 2 4 1 @@ -272,7 +272,7 @@ from numerics window w as (order by f_float4 range between 1 preceding and 1 following) -- !query schema -struct +struct -- !query output 1 -3.0 1 1 2 -1.0 2 3 @@ -289,7 +289,7 @@ from numerics window w as (order by f_float4 range between 1 preceding and 1.1 following) -- !query schema -struct +struct -- !query output 1 -3.0 1 1 2 -1.0 2 3 @@ -306,7 +306,7 @@ from numerics window w as (order by f_float4 range between 'inf' preceding and 'inf' following) -- !query schema -struct +struct -- !query output 1 -3.0 1 7 2 -1.0 1 7 @@ -323,7 +323,7 @@ from numerics window w as (order by f_float4 range between 1.1 preceding and 'NaN' following) -- !query schema -struct +struct -- !query output 1 -3.0 1 7 2 -1.0 2 7 @@ -340,7 +340,7 @@ from numerics window w as (order by f_float8 range between 1 preceding and 1 following) -- !query schema -struct +struct -- !query output 1 -3.0 1 1 2 -1.0 2 3 @@ -357,7 +357,7 @@ from numerics window w as (order by f_float8 range between 1 preceding and 1.1 following) -- !query schema -struct +struct -- !query output 1 -3.0 1 1 2 -1.0 2 3 @@ -374,7 +374,7 @@ from numerics window w as (order by f_float8 range between 'inf' preceding and 'inf' following) -- !query schema -struct +struct -- !query output 1 -3.0 1 7 2 -1.0 1 7 @@ -391,7 +391,7 @@ from numerics window w as (order by f_float8 range between 1.1 preceding and 'NaN' following) -- !query schema -struct +struct -- !query output 1 -3.0 1 7 2 -1.0 2 7 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out index acce68809257..121aa14aeadf 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out @@ -112,7 +112,7 @@ SELECT x, (sum(x) over w) FROM cte WINDOW w AS (ORDER BY x range between 1 preceding and 1 following) -- !query schema -struct +struct -- !query output 1 1 11 11 @@ -182,7 +182,7 @@ SELECT x, (sum(x) over w) FROM cte WINDOW w AS (ORDER BY x range between 1 preceding and 1 following) -- !query schema -struct +struct -- !query output 1 3 1 3 diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/promoteStrings.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/promoteStrings.sql.out index b8c190beeae1..af9c07b300db 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/promoteStrings.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/promoteStrings.sql.out @@ -2532,7 +2532,7 @@ NaN -- !query SELECT - '1' FROM t -- !query schema -struct<(- CAST(1 AS DOUBLE)):double> +struct<-(CAST(1 AS DOUBLE)):double> -- !query output -1.0 From 2cd091e9466fa555a5b8f7ad1acecc1d4a4f69f3 Mon Sep 17 00:00:00 2001 From: beliefer Date: Fri, 1 May 2020 09:52:41 +0800 Subject: [PATCH 4/5] Update golden file --- sql/core/src/test/resources/sql-tests/results/operators.sql.out | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/resources/sql-tests/results/operators.sql.out b/sql/core/src/test/resources/sql-tests/results/operators.sql.out index cf857cf9f98a..a94a123b1b8a 100644 --- a/sql/core/src/test/resources/sql-tests/results/operators.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/operators.sql.out @@ -437,7 +437,7 @@ struct -- !query select positive('-1.11'), positive(-1.11), negative('-1.11'), negative(-1.11) -- !query schema -struct<(+ CAST(-1.11 AS DOUBLE)):double,(+ -1.11):decimal(3,2),(- CAST(-1.11 AS DOUBLE)):double,(- -1.11):decimal(3,2)> +struct<(+ CAST(-1.11 AS DOUBLE)):double,(+ -1.11):decimal(3,2),negative(CAST(-1.11 AS DOUBLE)):double,negative(-1.11):decimal(3,2)> -- !query output -1.11 -1.11 1.11 1.11 From f048601421d12ada878880c6a71f43b1bff3adf3 Mon Sep 17 00:00:00 2001 From: beliefer Date: Fri, 1 May 2020 13:11:13 +0800 Subject: [PATCH 5/5] Fix typo. --- .../src/test/resources/sql-functions/sql-expression-schema.md | 2 +- .../scala/org/apache/spark/sql/ExpressionsSchemaSuite.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) 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 index 3ef82f513057..b43deede9705 100644 --- a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md +++ b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md @@ -1,4 +1,4 @@ - + ## Summary - Number of queries: 333 - Number of expressions that missing example: 34 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 index dd72473f0ea6..ac12e3f15dfa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ExpressionsSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExpressionsSchemaSuite.scala @@ -139,7 +139,7 @@ class ExpressionsSchemaSuite extends QueryTest with SharedSparkSession { if (regenerateGoldenFiles) { val missingExampleStr = missingExamples.mkString(",") val goldenOutput = { - s"\n" + + s"\n" + "## Summary\n" + s" - Number of queries: ${outputs.size}\n" + s" - Number of expressions that missing example: ${missingExamples.size}\n" +