Skip to content
Closed
Show file tree
Hide file tree
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Seems like you can remove this and replace it to prettyName.

Copy link
Contributor Author

@beliefer beliefer Apr 14, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes. I will replace it.
But wait for #28194 merged.
Then I could update golden file.


override protected def predicate(oldExpr: Expression, newExpr: Expression): Expression =
Expand All @@ -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 =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,8 @@ case class UnaryMinus(child: Expression) extends UnaryExpression

override def toString: String = s"-$child"

override def prettyName: String = "-"
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@HyukjinKwon Should we change "-" to "negative" ?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think it should be negative but should show - when it's used as the sql operator. You should be able to do this with if-else. e.g.)

getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse("-")

You might need to override sql IIRC.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

OK.


private lazy val numeric = TypeUtils.getNumeric(dataType, checkOverflow)

override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = dataType match {
Expand Down Expand Up @@ -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(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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)")
Expand All @@ -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})"
}
Original file line number Diff line number Diff line change
Expand Up @@ -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") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -98,15 +98,15 @@ 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


-- !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

Expand Down Expand Up @@ -689,7 +689,7 @@ select
interval '2-2' year to month + dateval
from interval_arithmetic
-- !query schema
struct<dateval:date,CAST(CAST(dateval AS TIMESTAMP) - INTERVAL '2 years 2 months' AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) - INTERVAL '-2 years -2 months' AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + INTERVAL '2 years 2 months' AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + INTERVAL '-2 years -2 months' AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + (- INTERVAL '2 years 2 months') AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + INTERVAL '2 years 2 months' AS DATE):date>
struct<dateval:date,CAST(CAST(dateval AS TIMESTAMP) - INTERVAL '2 years 2 months' AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) - INTERVAL '-2 years -2 months' AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + INTERVAL '2 years 2 months' AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + INTERVAL '-2 years -2 months' AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + -(INTERVAL '2 years 2 months') AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + INTERVAL '2 years 2 months' AS DATE):date>
-- !query output
2012-01-01 2009-11-01 2014-03-01 2014-03-01 2009-11-01 2009-11-01 2014-03-01

Expand All @@ -705,7 +705,7 @@ select
interval '2-2' year to month + tsval
from interval_arithmetic
-- !query schema
struct<tsval:timestamp,CAST(tsval - INTERVAL '2 years 2 months' AS TIMESTAMP):timestamp,CAST(tsval - INTERVAL '-2 years -2 months' AS TIMESTAMP):timestamp,CAST(tsval + INTERVAL '2 years 2 months' AS TIMESTAMP):timestamp,CAST(tsval + INTERVAL '-2 years -2 months' AS TIMESTAMP):timestamp,CAST(tsval + (- INTERVAL '2 years 2 months') AS TIMESTAMP):timestamp,CAST(tsval + INTERVAL '2 years 2 months' AS TIMESTAMP):timestamp>
struct<tsval:timestamp,CAST(tsval - INTERVAL '2 years 2 months' AS TIMESTAMP):timestamp,CAST(tsval - INTERVAL '-2 years -2 months' AS TIMESTAMP):timestamp,CAST(tsval + INTERVAL '2 years 2 months' AS TIMESTAMP):timestamp,CAST(tsval + INTERVAL '-2 years -2 months' AS TIMESTAMP):timestamp,CAST(tsval + -(INTERVAL '2 years 2 months') AS TIMESTAMP):timestamp,CAST(tsval + INTERVAL '2 years 2 months' AS TIMESTAMP):timestamp>
-- !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

Expand All @@ -732,7 +732,7 @@ select
interval '99 11:22:33.123456789' day to second + dateval
from interval_arithmetic
-- !query schema
struct<dateval:date,CAST(CAST(dateval AS TIMESTAMP) - INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) - INTERVAL '-99 days -11 hours -22 minutes -33.123456 seconds' AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + INTERVAL '-99 days -11 hours -22 minutes -33.123456 seconds' AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + (- INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds') AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' AS DATE):date>
struct<dateval:date,CAST(CAST(dateval AS TIMESTAMP) - INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) - INTERVAL '-99 days -11 hours -22 minutes -33.123456 seconds' AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + INTERVAL '-99 days -11 hours -22 minutes -33.123456 seconds' AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + -(INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds') AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' AS DATE):date>
-- !query output
2012-01-01 2011-09-23 2012-04-09 2012-04-09 2011-09-23 2011-09-23 2012-04-09

Expand All @@ -748,7 +748,7 @@ select
interval '99 11:22:33.123456789' day to second + tsval
from interval_arithmetic
-- !query schema
struct<tsval:timestamp,CAST(tsval - INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' AS TIMESTAMP):timestamp,CAST(tsval - INTERVAL '-99 days -11 hours -22 minutes -33.123456 seconds' AS TIMESTAMP):timestamp,CAST(tsval + INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' AS TIMESTAMP):timestamp,CAST(tsval + INTERVAL '-99 days -11 hours -22 minutes -33.123456 seconds' AS TIMESTAMP):timestamp,CAST(tsval + (- INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds') AS TIMESTAMP):timestamp,CAST(tsval + INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' AS TIMESTAMP):timestamp>
struct<tsval:timestamp,CAST(tsval - INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' AS TIMESTAMP):timestamp,CAST(tsval - INTERVAL '-99 days -11 hours -22 minutes -33.123456 seconds' AS TIMESTAMP):timestamp,CAST(tsval + INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' AS TIMESTAMP):timestamp,CAST(tsval + INTERVAL '-99 days -11 hours -22 minutes -33.123456 seconds' AS TIMESTAMP):timestamp,CAST(tsval + -(INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds') AS TIMESTAMP):timestamp,CAST(tsval + INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' AS TIMESTAMP):timestamp>
-- !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

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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
Expand All @@ -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
Expand Down
14 changes: 7 additions & 7 deletions sql/core/src/test/resources/sql-tests/results/interval.sql.out
Original file line number Diff line number Diff line change
Expand Up @@ -93,15 +93,15 @@ 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


-- !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

Expand Down Expand Up @@ -668,7 +668,7 @@ select
interval '2-2' year to month + dateval
from interval_arithmetic
-- !query schema
struct<dateval:date,CAST(CAST(dateval AS TIMESTAMP) - INTERVAL '2 years 2 months' AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) - INTERVAL '-2 years -2 months' AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + INTERVAL '2 years 2 months' AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + INTERVAL '-2 years -2 months' AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + (- INTERVAL '2 years 2 months') AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + INTERVAL '2 years 2 months' AS DATE):date>
struct<dateval:date,CAST(CAST(dateval AS TIMESTAMP) - INTERVAL '2 years 2 months' AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) - INTERVAL '-2 years -2 months' AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + INTERVAL '2 years 2 months' AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + INTERVAL '-2 years -2 months' AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + -(INTERVAL '2 years 2 months') AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + INTERVAL '2 years 2 months' AS DATE):date>
-- !query output
2012-01-01 2009-11-01 2014-03-01 2014-03-01 2009-11-01 2009-11-01 2014-03-01

Expand All @@ -684,7 +684,7 @@ select
interval '2-2' year to month + tsval
from interval_arithmetic
-- !query schema
struct<tsval:timestamp,CAST(tsval - INTERVAL '2 years 2 months' AS TIMESTAMP):timestamp,CAST(tsval - INTERVAL '-2 years -2 months' AS TIMESTAMP):timestamp,CAST(tsval + INTERVAL '2 years 2 months' AS TIMESTAMP):timestamp,CAST(tsval + INTERVAL '-2 years -2 months' AS TIMESTAMP):timestamp,CAST(tsval + (- INTERVAL '2 years 2 months') AS TIMESTAMP):timestamp,CAST(tsval + INTERVAL '2 years 2 months' AS TIMESTAMP):timestamp>
struct<tsval:timestamp,CAST(tsval - INTERVAL '2 years 2 months' AS TIMESTAMP):timestamp,CAST(tsval - INTERVAL '-2 years -2 months' AS TIMESTAMP):timestamp,CAST(tsval + INTERVAL '2 years 2 months' AS TIMESTAMP):timestamp,CAST(tsval + INTERVAL '-2 years -2 months' AS TIMESTAMP):timestamp,CAST(tsval + -(INTERVAL '2 years 2 months') AS TIMESTAMP):timestamp,CAST(tsval + INTERVAL '2 years 2 months' AS TIMESTAMP):timestamp>
-- !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

Expand All @@ -711,7 +711,7 @@ select
interval '99 11:22:33.123456789' day to second + dateval
from interval_arithmetic
-- !query schema
struct<dateval:date,CAST(CAST(dateval AS TIMESTAMP) - INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) - INTERVAL '-99 days -11 hours -22 minutes -33.123456 seconds' AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + INTERVAL '-99 days -11 hours -22 minutes -33.123456 seconds' AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + (- INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds') AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' AS DATE):date>
struct<dateval:date,CAST(CAST(dateval AS TIMESTAMP) - INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) - INTERVAL '-99 days -11 hours -22 minutes -33.123456 seconds' AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + INTERVAL '-99 days -11 hours -22 minutes -33.123456 seconds' AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + -(INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds') AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' AS DATE):date>
-- !query output
2012-01-01 2011-09-23 2012-04-09 2012-04-09 2011-09-23 2011-09-23 2012-04-09

Expand All @@ -727,7 +727,7 @@ select
interval '99 11:22:33.123456789' day to second + tsval
from interval_arithmetic
-- !query schema
struct<tsval:timestamp,CAST(tsval - INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' AS TIMESTAMP):timestamp,CAST(tsval - INTERVAL '-99 days -11 hours -22 minutes -33.123456 seconds' AS TIMESTAMP):timestamp,CAST(tsval + INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' AS TIMESTAMP):timestamp,CAST(tsval + INTERVAL '-99 days -11 hours -22 minutes -33.123456 seconds' AS TIMESTAMP):timestamp,CAST(tsval + (- INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds') AS TIMESTAMP):timestamp,CAST(tsval + INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' AS TIMESTAMP):timestamp>
struct<tsval:timestamp,CAST(tsval - INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' AS TIMESTAMP):timestamp,CAST(tsval - INTERVAL '-99 days -11 hours -22 minutes -33.123456 seconds' AS TIMESTAMP):timestamp,CAST(tsval + INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' AS TIMESTAMP):timestamp,CAST(tsval + INTERVAL '-99 days -11 hours -22 minutes -33.123456 seconds' AS TIMESTAMP):timestamp,CAST(tsval + -(INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds') AS TIMESTAMP):timestamp,CAST(tsval + INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' AS TIMESTAMP):timestamp>
-- !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

Expand Down Expand Up @@ -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

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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
Expand All @@ -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
Expand Down
22 changes: 11 additions & 11 deletions sql/core/src/test/resources/sql-tests/results/operators.sql.out
Original file line number Diff line number Diff line change
Expand Up @@ -37,63 +37,63 @@ 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


-- !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


-- !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


-- !query
select - (-10)
-- !query schema
struct<(- -10):int>
struct<-(-10):int>
-- !query output
10


-- !query
select + (-key) from testdata where key = 32
-- !query schema
struct<(+ (- key)):int>
struct<(+ -(key)):int>
-- !query output
-32


-- !query
select - (+max(key)) from testdata
-- !query schema
struct<(- (+ max(key))):int>
struct<-((+ max(key))):int>
-- !query output
-100


-- !query
select - - 3
-- !query schema
struct<(- -3):int>
struct<-(-3):int>
-- !query output
3


-- !query
select - + 20
-- !query schema
struct<(- (+ 20)):int>
struct<-((+ 20)):int>
-- !query output
-20

Expand All @@ -109,15 +109,15 @@ struct<(+ (+ 100)):int>
-- !query
select - - max(key) from testdata
-- !query schema
struct<(- (- max(key))):int>
struct<-(-(max(key))):int>
-- !query output
100


-- !query
select + - key from testdata where key = 33
-- !query schema
struct<(+ (- key)):int>
struct<(+ -(key)):int>
-- !query output
-33

Expand Down Expand Up @@ -437,7 +437,7 @@ struct<abs(-3.13):decimal(3,2),abs(CAST(-2.19 AS DOUBLE)):double>
-- !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

Expand Down
Loading