diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 625ef2153c71..2b054b5b8b6c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -228,6 +228,7 @@ class Analyzer( ResolveLambdaVariables(conf) :: ResolveTimeZone(conf) :: ResolveRandomSeed :: + ResolveBinaryArithmetic(conf) :: TypeCoercion.typeCoercionRules(conf) ++ extendedResolutionRules : _*), Batch("PostgreSQL Dialect", Once, PostgreSQLDialect.postgreSQLDialectRules: _*), @@ -246,6 +247,65 @@ class Analyzer( CleanupAliases) ) + /** + * For [[Add]]: + * 1. if both side are interval, stays the same; + * 2. else if one side is interval, turns it to [[TimeAdd]]; + * 3. else if one side is date, turns it to [[DateAdd]] ; + * 4. else stays the same. + * + * For [[Subtract]]: + * 1. if both side are interval, stays the same; + * 2. else if the right side is an interval, turns it to [[TimeSub]]; + * 3. else if one side is timestamp, turns it to [[SubtractTimestamps]]; + * 4. else if the right side is date, turns it to [[DateDiff]]/[[SubtractDates]]; + * 5. else if the left side is date, turns it to [[DateSub]]; + * 6. else turns it to stays the same. + * + * For [[Multiply]]: + * 1. If one side is interval, turns it to [[MultiplyInterval]]; + * 2. otherwise, stays the same. + * + * For [[Divide]]: + * 1. If the left side is interval, turns it to [[DivideInterval]]; + * 2. otherwise, stays the same. + */ + case class ResolveBinaryArithmetic(conf: SQLConf) extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { + case p: LogicalPlan => p.transformExpressionsUp { + case a @ Add(l, r) if a.childrenResolved => (l.dataType, r.dataType) match { + case (CalendarIntervalType, CalendarIntervalType) => a + case (_, CalendarIntervalType) => Cast(TimeAdd(l, r), l.dataType) + case (CalendarIntervalType, _) => Cast(TimeAdd(r, l), r.dataType) + case (DateType, _) => DateAdd(l, r) + case (_, DateType) => DateAdd(r, l) + case _ => a + } + case s @ Subtract(l, r) if s.childrenResolved => (l.dataType, r.dataType) match { + case (CalendarIntervalType, CalendarIntervalType) => s + case (_, CalendarIntervalType) => Cast(TimeSub(l, r), l.dataType) + case (TimestampType, _) => SubtractTimestamps(l, r) + case (_, TimestampType) => SubtractTimestamps(l, r) + case (_, DateType) => if (conf.usePostgreSQLDialect) { + DateDiff(l, r) + } else { + SubtractDates(l, r) + } + case (DateType, _) => DateSub(l, r) + case _ => s + } + case m @ Multiply(l, r) if m.childrenResolved => (l.dataType, r.dataType) match { + case (CalendarIntervalType, _) => MultiplyInterval(l, r) + case (_, CalendarIntervalType) => MultiplyInterval(r, l) + case _ => m + } + case d @ Divide(l, r) if d.childrenResolved => (l.dataType, r.dataType) match { + case (CalendarIntervalType, _) => DivideInterval(l, r) + case _ => d + } + } + } + } /** * Substitute child plan with WindowSpecDefinitions. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala index 83c76c2d4e2b..830fa5383bdc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala @@ -822,52 +822,24 @@ object TypeCoercion { } } - /** - * 1. Turns Add/Subtract of DateType/TimestampType/StringType and CalendarIntervalType - * to TimeAdd/TimeSub. - * 2. Turns Add/Subtract of TimestampType/DateType/IntegerType - * and TimestampType/IntegerType/DateType to DateAdd/DateSub/SubtractDates and - * to SubtractTimestamps. - * 3. Turns Multiply/Divide of CalendarIntervalType and NumericType - * to MultiplyInterval/DivideInterval - */ object DateTimeOperations extends Rule[LogicalPlan] { - - private val acceptedTypes = Seq(DateType, TimestampType, StringType) - - def apply(plan: LogicalPlan): LogicalPlan = plan resolveExpressions { + override def apply(plan: LogicalPlan): LogicalPlan = plan resolveExpressions { // Skip nodes who's children have not been resolved yet. case e if !e.childrenResolved => e - - case Add(l @ CalendarIntervalType(), r) if acceptedTypes.contains(r.dataType) => - Cast(TimeAdd(r, l), r.dataType) - case Add(l, r @ CalendarIntervalType()) if acceptedTypes.contains(l.dataType) => - Cast(TimeAdd(l, r), l.dataType) - case Subtract(l, r @ CalendarIntervalType()) if acceptedTypes.contains(l.dataType) => - Cast(TimeSub(l, r), l.dataType) - case Multiply(l @ CalendarIntervalType(), r @ NumericType()) => - MultiplyInterval(l, r) - case Multiply(l @ NumericType(), r @ CalendarIntervalType()) => - MultiplyInterval(r, l) - case Divide(l @ CalendarIntervalType(), r @ NumericType()) => - DivideInterval(l, r) - - case b @ BinaryOperator(l @ CalendarIntervalType(), r @ NullType()) => - b.withNewChildren(Seq(l, Cast(r, CalendarIntervalType))) - case b @ BinaryOperator(l @ NullType(), r @ CalendarIntervalType()) => - b.withNewChildren(Seq(Cast(l, CalendarIntervalType), r)) - - case Add(l @ DateType(), r @ IntegerType()) => DateAdd(l, r) - case Add(l @ IntegerType(), r @ DateType()) => DateAdd(r, l) - case Subtract(l @ DateType(), r @ IntegerType()) => DateSub(l, r) - case Subtract(l @ DateType(), r @ DateType()) => - if (SQLConf.get.usePostgreSQLDialect) DateDiff(l, r) else SubtractDates(l, r) - case Subtract(l @ TimestampType(), r @ TimestampType()) => - SubtractTimestamps(l, r) - case Subtract(l @ TimestampType(), r @ DateType()) => - SubtractTimestamps(l, Cast(r, TimestampType)) - case Subtract(l @ DateType(), r @ TimestampType()) => - SubtractTimestamps(Cast(l, TimestampType), r) + case d @ DateAdd(TimestampType(), _) => d.copy(startDate = Cast(d.startDate, DateType)) + case d @ DateAdd(StringType(), _) => d.copy(startDate = Cast(d.startDate, DateType)) + case d @ DateSub(TimestampType(), _) => d.copy(startDate = Cast(d.startDate, DateType)) + case d @ DateSub(StringType(), _) => d.copy(startDate = Cast(d.startDate, DateType)) + + case s @ SubtractTimestamps(DateType(), _) => + s.copy(endTimestamp = Cast(s.endTimestamp, TimestampType)) + case s @ SubtractTimestamps(_, DateType()) => + s.copy(startTimestamp = Cast(s.startTimestamp, TimestampType)) + + case t @ TimeAdd(DateType(), _, _) => t.copy(start = Cast(t.start, TimestampType)) + case t @ TimeAdd(StringType(), _, _) => t.copy(start = Cast(t.start, TimestampType)) + case t @ TimeSub(DateType(), _, _) => t.copy(start = Cast(t.start, TimestampType)) + case t @ TimeSub(StringType(), _, _) => t.copy(start = Cast(t.start, TimestampType)) } } @@ -881,11 +853,8 @@ object TypeCoercion { case e if !e.childrenResolved => e // If DecimalType operands are involved, DecimalPrecision will handle it - // If CalendarIntervalType operands are involved, DateTimeOperations will handle it case b @ BinaryOperator(left, right) if !left.dataType.isInstanceOf[DecimalType] && !right.dataType.isInstanceOf[DecimalType] && - !left.dataType.isInstanceOf[CalendarIntervalType] && - !right.dataType.isInstanceOf[CalendarIntervalType] && left.dataType != right.dataType => findTightestCommonType(left.dataType, right.dataType).map { commonType => if (b.inputType.acceptsType(commonType)) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index f5023a10baa6..b3aeda852a4d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -151,17 +151,18 @@ case class CurrentBatchTimestamp( """, since = "1.5.0") case class DateAdd(startDate: Expression, days: Expression) - extends BinaryExpression with ImplicitCastInputTypes { + extends BinaryExpression with ExpectsInputTypes { override def left: Expression = startDate override def right: Expression = days - override def inputTypes: Seq[AbstractDataType] = Seq(DateType, IntegerType) + override def inputTypes: Seq[AbstractDataType] = + Seq(DateType, TypeCollection(IntegerType, ShortType, ByteType)) override def dataType: DataType = DateType override def nullSafeEval(start: Any, d: Any): Any = { - start.asInstanceOf[Int] + d.asInstanceOf[Int] + start.asInstanceOf[Int] + d.asInstanceOf[Number].intValue() } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { @@ -185,16 +186,17 @@ case class DateAdd(startDate: Expression, days: Expression) """, since = "1.5.0") case class DateSub(startDate: Expression, days: Expression) - extends BinaryExpression with ImplicitCastInputTypes { + extends BinaryExpression with ExpectsInputTypes { override def left: Expression = startDate override def right: Expression = days - override def inputTypes: Seq[AbstractDataType] = Seq(DateType, IntegerType) + override def inputTypes: Seq[AbstractDataType] = + Seq(DateType, TypeCollection(IntegerType, ShortType, ByteType)) override def dataType: DataType = DateType override def nullSafeEval(start: Any, d: Any): Any = { - start.asInstanceOf[Int] - d.asInstanceOf[Int] + start.asInstanceOf[Int] - d.asInstanceOf[Number].intValue() } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { @@ -1072,7 +1074,7 @@ case class NextDay(startDate: Expression, dayOfWeek: Expression) * Adds an interval to timestamp. */ case class TimeAdd(start: Expression, interval: Expression, timeZoneId: Option[String] = None) - extends BinaryExpression with TimeZoneAwareExpression with ImplicitCastInputTypes { + extends BinaryExpression with TimeZoneAwareExpression with ExpectsInputTypes { def this(start: Expression, interval: Expression) = this(start, interval, None) @@ -1187,7 +1189,7 @@ case class FromUTCTimestamp(left: Expression, right: Expression) * Subtracts an interval from timestamp. */ case class TimeSub(start: Expression, interval: Expression, timeZoneId: Option[String] = None) - extends BinaryExpression with TimeZoneAwareExpression with ImplicitCastInputTypes { + extends BinaryExpression with TimeZoneAwareExpression with ExpectsInputTypes { def this(start: Expression, interval: Expression) = this(start, interval, None) @@ -2127,7 +2129,7 @@ case class DatePart(field: Expression, source: Expression, child: Expression) * between the given timestamps. */ case class SubtractTimestamps(endTimestamp: Expression, startTimestamp: Expression) - extends BinaryExpression with ImplicitCastInputTypes { + extends BinaryExpression with ExpectsInputTypes { override def left: Expression = endTimestamp override def right: Expression = startTimestamp diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala index 567cf5ec8ebe..f35617b374c9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala @@ -1401,44 +1401,6 @@ class TypeCoercionSuite extends AnalysisTest { } } - test("rule for date/timestamp operations") { - val dateTimeOperations = TypeCoercion.DateTimeOperations - val date = Literal(new java.sql.Date(0L)) - val timestamp = Literal(new Timestamp(0L)) - val interval = Literal(new CalendarInterval(0, 0, 0)) - val str = Literal("2015-01-01") - val intValue = Literal(0, IntegerType) - - ruleTest(dateTimeOperations, Add(date, interval), Cast(TimeAdd(date, interval), DateType)) - ruleTest(dateTimeOperations, Add(interval, date), Cast(TimeAdd(date, interval), DateType)) - ruleTest(dateTimeOperations, Add(timestamp, interval), - Cast(TimeAdd(timestamp, interval), TimestampType)) - ruleTest(dateTimeOperations, Add(interval, timestamp), - Cast(TimeAdd(timestamp, interval), TimestampType)) - ruleTest(dateTimeOperations, Add(str, interval), Cast(TimeAdd(str, interval), StringType)) - ruleTest(dateTimeOperations, Add(interval, str), Cast(TimeAdd(str, interval), StringType)) - - ruleTest(dateTimeOperations, Subtract(date, interval), Cast(TimeSub(date, interval), DateType)) - ruleTest(dateTimeOperations, Subtract(timestamp, interval), - Cast(TimeSub(timestamp, interval), TimestampType)) - ruleTest(dateTimeOperations, Subtract(str, interval), Cast(TimeSub(str, interval), StringType)) - - // interval operations should not be effected - ruleTest(dateTimeOperations, Add(interval, interval), Add(interval, interval)) - ruleTest(dateTimeOperations, Subtract(interval, interval), Subtract(interval, interval)) - - ruleTest(dateTimeOperations, Add(date, intValue), DateAdd(date, intValue)) - ruleTest(dateTimeOperations, Add(intValue, date), DateAdd(date, intValue)) - ruleTest(dateTimeOperations, Subtract(date, intValue), DateSub(date, intValue)) - ruleTest(dateTimeOperations, Subtract(date, date), SubtractDates(date, date)) - ruleTest(dateTimeOperations, Subtract(timestamp, timestamp), - SubtractTimestamps(timestamp, timestamp)) - ruleTest(dateTimeOperations, Subtract(timestamp, date), - SubtractTimestamps(timestamp, Cast(date, TimestampType))) - ruleTest(dateTimeOperations, Subtract(date, timestamp), - SubtractTimestamps(Cast(date, TimestampType), timestamp)) - } - /** * There are rules that need to not fire before child expressions get resolved. * We use this test to make sure those rules do not fire early. @@ -1586,27 +1548,6 @@ class TypeCoercionSuite extends AnalysisTest { Multiply(CaseWhen(Seq((EqualTo(1, 2), Cast(1, DecimalType(34, 24)))), Cast(100, DecimalType(34, 24))), Cast(1, IntegerType))) } - - test("rule for interval operations") { - val dateTimeOperations = TypeCoercion.DateTimeOperations - val interval = Literal(new CalendarInterval(0, 0, 0)) - - Seq( - Literal(10.toByte, ByteType), - Literal(10.toShort, ShortType), - Literal(10, IntegerType), - Literal(10L, LongType), - Literal(Decimal(10), DecimalType.SYSTEM_DEFAULT), - Literal(10.5.toFloat, FloatType), - Literal(10.5, DoubleType)).foreach { num => - ruleTest(dateTimeOperations, Multiply(interval, num), - MultiplyInterval(interval, num)) - ruleTest(dateTimeOperations, Multiply(num, interval), - MultiplyInterval(interval, num)) - ruleTest(dateTimeOperations, Divide(interval, num), - DivideInterval(interval, num)) - } - } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index 5cd4d11e32f7..7d015aa478df 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -332,6 +332,12 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("date_add") { + checkEvaluation( + DateAdd(Literal(Date.valueOf("2016-02-28")), Literal(1.toByte)), + DateTimeUtils.fromJavaDate(Date.valueOf("2016-02-29"))) + checkEvaluation( + DateAdd(Literal(Date.valueOf("2016-02-28")), Literal(1.toShort)), + DateTimeUtils.fromJavaDate(Date.valueOf("2016-02-29"))) checkEvaluation( DateAdd(Literal(Date.valueOf("2016-02-28")), Literal(1)), DateTimeUtils.fromJavaDate(Date.valueOf("2016-02-29"))) @@ -347,10 +353,18 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { DateAdd(Literal(Date.valueOf("2016-02-28")), positiveIntLit), 49627) checkEvaluation( DateAdd(Literal(Date.valueOf("2016-02-28")), negativeIntLit), -15910) + checkConsistencyBetweenInterpretedAndCodegen(DateAdd, DateType, ByteType) + checkConsistencyBetweenInterpretedAndCodegen(DateAdd, DateType, ShortType) checkConsistencyBetweenInterpretedAndCodegen(DateAdd, DateType, IntegerType) } test("date_sub") { + checkEvaluation( + DateSub(Literal(Date.valueOf("2015-01-01")), Literal(1.toByte)), + DateTimeUtils.fromJavaDate(Date.valueOf("2014-12-31"))) + checkEvaluation( + DateSub(Literal(Date.valueOf("2015-01-01")), Literal(1.toShort)), + DateTimeUtils.fromJavaDate(Date.valueOf("2014-12-31"))) checkEvaluation( DateSub(Literal(Date.valueOf("2015-01-01")), Literal(1)), DateTimeUtils.fromJavaDate(Date.valueOf("2014-12-31"))) @@ -366,6 +380,8 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { DateSub(Literal(Date.valueOf("2016-02-28")), positiveIntLit), -15909) checkEvaluation( DateSub(Literal(Date.valueOf("2016-02-28")), negativeIntLit), 49628) + checkConsistencyBetweenInterpretedAndCodegen(DateSub, DateType, ByteType) + checkConsistencyBetweenInterpretedAndCodegen(DateSub, DateType, ShortType) checkConsistencyBetweenInterpretedAndCodegen(DateSub, DateType, IntegerType) } diff --git a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql index de2040c65123..b14778b91510 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql @@ -30,12 +30,48 @@ select weekday('2007-02-03'), weekday('2009-07-30'), weekday('2017-05-27'), week select year('1500-01-01'), month('1500-01-01'), dayOfYear('1500-01-01'); -select date '2001-09-28' + 7; -select 7 + date '2001-09-28'; -select date '2001-10-01' - 7; -select date '2001-10-01' - date '2001-09-28'; -select date'2020-01-01' - timestamp'2019-10-06 10:11:12.345678'; -select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01'; select date '2019-01-01\t'; select timestamp '2019-01-01\t'; + +-- time add/sub +select timestamp'2011-11-11 11:11:11' + interval '2' day; +select timestamp'2011-11-11 11:11:11' - interval '2' day; +select date'2011-11-11 11:11:11' + interval '2' second; +select date'2011-11-11 11:11:11' - interval '2' second; +select '2011-11-11' - interval '2' day; +select '2011-11-11 11:11:11' - interval '2' second; +select '1' - interval '2' second; +select 1 - interval '2' second; + +-- subtract timestamps +select date'2020-01-01' - timestamp'2019-10-06 10:11:12.345678'; +select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01'; +select timestamp'2019-10-06 10:11:12.345678' - null; +select null - timestamp'2019-10-06 10:11:12.345678'; + +-- date add/sub +select date_add('2011-11-11', 1Y); +select date_add('2011-11-11', 1S); +select date_add('2011-11-11', 1); +select date_add('2011-11-11', 1L); +select date_add('2011-11-11', 1.0); +select date_add('2011-11-11', 1E1); +select date_add('2011-11-11', '1'); +select date_add(date'2011-11-11', 1); +select date_add(timestamp'2011-11-11', 1); +select date_sub(date'2011-11-11', 1); +select date_sub(timestamp'2011-11-11', 1); +select date_sub(null, 1); +select date_sub(date'2011-11-11', null); +select date'2011-11-11' + 1E1; +select null + date '2001-09-28'; +select date '2001-09-28' + 7Y; +select 7S + date '2001-09-28'; +select date '2001-10-01' - 7; +select date '2001-09-28' + null; +select date '2001-09-28' - null; + +-- subtract dates +select null - date '2019-10-06'; +select date '2001-10-01' - date '2001-09-28'; diff --git a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out index a269420a9a8b..09584550e3f4 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 19 +-- Number of queries: 47 -- !query 0 @@ -100,64 +100,294 @@ struct +struct -- !query 11 output -2001-10-05 +2019-01-01 -- !query 12 -select 7 + date '2001-09-28' +select timestamp '2019-01-01\t' -- !query 12 schema -struct +struct -- !query 12 output -2001-10-05 +2019-01-01 00:00:00 -- !query 13 -select date '2001-10-01' - 7 +select timestamp'2011-11-11 11:11:11' + interval '2' day -- !query 13 schema -struct +struct -- !query 13 output -2001-09-24 +2011-11-13 11:11:11 -- !query 14 -select date '2001-10-01' - date '2001-09-28' +select timestamp'2011-11-11 11:11:11' - interval '2' day -- !query 14 schema -struct +struct -- !query 14 output -3 days +2011-11-09 11:11:11 -- !query 15 -select date'2020-01-01' - timestamp'2019-10-06 10:11:12.345678' +select date'2011-11-11 11:11:11' + interval '2' second -- !query 15 schema -struct +struct -- !query 15 output -2078 hours 48 minutes 47.654322 seconds +2011-11-11 -- !query 16 -select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01' +select date'2011-11-11 11:11:11' - interval '2' second -- !query 16 schema -struct +struct -- !query 16 output --2078 hours -48 minutes -47.654322 seconds +2011-11-10 -- !query 17 -select date '2019-01-01\t' +select '2011-11-11' - interval '2' day -- !query 17 schema -struct +struct -- !query 17 output -2019-01-01 +2011-11-09 00:00:00 -- !query 18 -select timestamp '2019-01-01\t' +select '2011-11-11 11:11:11' - interval '2' second -- !query 18 schema -struct +struct -- !query 18 output -2019-01-01 00:00:00 +2011-11-11 11:11:09 + + +-- !query 19 +select '1' - interval '2' second +-- !query 19 schema +struct +-- !query 19 output +NULL + + +-- !query 20 +select 1 - interval '2' second +-- !query 20 schema +struct<> +-- !query 20 output +org.apache.spark.sql.AnalysisException +cannot resolve '1 - INTERVAL '2 seconds'' due to data type mismatch: argument 1 requires timestamp type, however, '1' is of int type.; line 1 pos 7 + + +-- !query 21 +select date'2020-01-01' - timestamp'2019-10-06 10:11:12.345678' +-- !query 21 schema +struct +-- !query 21 output +2078 hours 48 minutes 47.654322 seconds + + +-- !query 22 +select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01' +-- !query 22 schema +struct +-- !query 22 output +-2078 hours -48 minutes -47.654322 seconds + + +-- !query 23 +select timestamp'2019-10-06 10:11:12.345678' - null +-- !query 23 schema +struct +-- !query 23 output +NULL + + +-- !query 24 +select null - timestamp'2019-10-06 10:11:12.345678' +-- !query 24 schema +struct +-- !query 24 output +NULL + + +-- !query 25 +select date_add('2011-11-11', 1Y) +-- !query 25 schema +struct +-- !query 25 output +2011-11-12 + + +-- !query 26 +select date_add('2011-11-11', 1S) +-- !query 26 schema +struct +-- !query 26 output +2011-11-12 + + +-- !query 27 +select date_add('2011-11-11', 1) +-- !query 27 schema +struct +-- !query 27 output +2011-11-12 + + +-- !query 28 +select date_add('2011-11-11', 1L) +-- !query 28 schema +struct<> +-- !query 28 output +org.apache.spark.sql.AnalysisException +cannot resolve 'date_add(CAST('2011-11-11' AS DATE), 1L)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '1L' is of bigint type.; line 1 pos 7 + + +-- !query 29 +select date_add('2011-11-11', 1.0) +-- !query 29 schema +struct<> +-- !query 29 output +org.apache.spark.sql.AnalysisException +cannot resolve 'date_add(CAST('2011-11-11' AS DATE), 1.0BD)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '1.0BD' is of decimal(2,1) type.; line 1 pos 7 + + +-- !query 30 +select date_add('2011-11-11', 1E1) +-- !query 30 schema +struct<> +-- !query 30 output +org.apache.spark.sql.AnalysisException +cannot resolve 'date_add(CAST('2011-11-11' AS DATE), 10.0D)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '10.0D' is of double type.; line 1 pos 7 + + +-- !query 31 +select date_add('2011-11-11', '1') +-- !query 31 schema +struct<> +-- !query 31 output +org.apache.spark.sql.AnalysisException +cannot resolve 'date_add(CAST('2011-11-11' AS DATE), '1')' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, ''1'' is of string type.; line 1 pos 7 + + +-- !query 32 +select date_add(date'2011-11-11', 1) +-- !query 32 schema +struct +-- !query 32 output +2011-11-12 + + +-- !query 33 +select date_add(timestamp'2011-11-11', 1) +-- !query 33 schema +struct +-- !query 33 output +2011-11-12 + + +-- !query 34 +select date_sub(date'2011-11-11', 1) +-- !query 34 schema +struct +-- !query 34 output +2011-11-10 + + +-- !query 35 +select date_sub(timestamp'2011-11-11', 1) +-- !query 35 schema +struct +-- !query 35 output +2011-11-10 + + +-- !query 36 +select date_sub(null, 1) +-- !query 36 schema +struct +-- !query 36 output +NULL + + +-- !query 37 +select date_sub(date'2011-11-11', null) +-- !query 37 schema +struct +-- !query 37 output +NULL + + +-- !query 38 +select date'2011-11-11' + 1E1 +-- !query 38 schema +struct<> +-- !query 38 output +org.apache.spark.sql.AnalysisException +cannot resolve 'date_add(DATE '2011-11-11', 10.0D)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '10.0D' is of double type.; line 1 pos 7 + + +-- !query 39 +select null + date '2001-09-28' +-- !query 39 schema +struct +-- !query 39 output +NULL + + +-- !query 40 +select date '2001-09-28' + 7Y +-- !query 40 schema +struct +-- !query 40 output +2001-10-05 + + +-- !query 41 +select 7S + date '2001-09-28' +-- !query 41 schema +struct +-- !query 41 output +2001-10-05 + + +-- !query 42 +select date '2001-10-01' - 7 +-- !query 42 schema +struct +-- !query 42 output +2001-09-24 + + +-- !query 43 +select date '2001-09-28' + null +-- !query 43 schema +struct +-- !query 43 output +NULL + + +-- !query 44 +select date '2001-09-28' - null +-- !query 44 schema +struct +-- !query 44 output +NULL + + +-- !query 45 +select null - date '2019-10-06' +-- !query 45 schema +struct +-- !query 45 output +NULL + + +-- !query 46 +select date '2001-10-01' - date '2001-09-28' +-- !query 46 schema +struct +-- !query 46 output +3 days diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/dateTimeOperations.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/dateTimeOperations.sql.out index 8608fa494d6c..07bdfe560a42 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/dateTimeOperations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/dateTimeOperations.sql.out @@ -16,7 +16,7 @@ select cast(1 as tinyint) + interval 2 day struct<> -- !query 1 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS TINYINT) + INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS TINYINT) + INTERVAL '2 days')' (tinyint and interval).; line 1 pos 7 +cannot resolve 'CAST(1 AS TINYINT) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS TINYINT)' is of tinyint type.; line 1 pos 7 -- !query 2 @@ -25,7 +25,7 @@ select cast(1 as smallint) + interval 2 day struct<> -- !query 2 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS SMALLINT) + INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS SMALLINT) + INTERVAL '2 days')' (smallint and interval).; line 1 pos 7 +cannot resolve 'CAST(1 AS SMALLINT) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS SMALLINT)' is of smallint type.; line 1 pos 7 -- !query 3 @@ -34,7 +34,7 @@ select cast(1 as int) + interval 2 day struct<> -- !query 3 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS INT) + INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS INT) + INTERVAL '2 days')' (int and interval).; line 1 pos 7 +cannot resolve 'CAST(1 AS INT) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS INT)' is of int type.; line 1 pos 7 -- !query 4 @@ -43,7 +43,7 @@ select cast(1 as bigint) + interval 2 day struct<> -- !query 4 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS BIGINT) + INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS BIGINT) + INTERVAL '2 days')' (bigint and interval).; line 1 pos 7 +cannot resolve 'CAST(1 AS BIGINT) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS BIGINT)' is of bigint type.; line 1 pos 7 -- !query 5 @@ -52,7 +52,7 @@ select cast(1 as float) + interval 2 day struct<> -- !query 5 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS FLOAT) + INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS FLOAT) + INTERVAL '2 days')' (float and interval).; line 1 pos 7 +cannot resolve 'CAST(1 AS FLOAT) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS FLOAT)' is of float type.; line 1 pos 7 -- !query 6 @@ -61,7 +61,7 @@ select cast(1 as double) + interval 2 day struct<> -- !query 6 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DOUBLE) + INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS DOUBLE) + INTERVAL '2 days')' (double and interval).; line 1 pos 7 +cannot resolve 'CAST(1 AS DOUBLE) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS DOUBLE)' is of double type.; line 1 pos 7 -- !query 7 @@ -70,7 +70,7 @@ select cast(1 as decimal(10, 0)) + interval 2 day struct<> -- !query 7 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DECIMAL(10,0)) + INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) + INTERVAL '2 days')' (decimal(10,0) and interval).; line 1 pos 7 +cannot resolve 'CAST(1 AS DECIMAL(10,0)) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS DECIMAL(10,0))' is of decimal(10,0) type.; line 1 pos 7 -- !query 8 @@ -95,7 +95,7 @@ select cast('1' as binary) + interval 2 day struct<> -- !query 10 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('1' AS BINARY) + INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST('1' AS BINARY) + INTERVAL '2 days')' (binary and interval).; line 1 pos 7 +cannot resolve 'CAST('1' AS BINARY) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST('1' AS BINARY)' is of binary type.; line 1 pos 7 -- !query 11 @@ -104,7 +104,7 @@ select cast(1 as boolean) + interval 2 day struct<> -- !query 11 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS BOOLEAN) + INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) + INTERVAL '2 days')' (boolean and interval).; line 1 pos 7 +cannot resolve 'CAST(1 AS BOOLEAN) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS BOOLEAN)' is of boolean type.; line 1 pos 7 -- !query 12 @@ -129,7 +129,7 @@ select interval 2 day + cast(1 as tinyint) struct<> -- !query 14 output org.apache.spark.sql.AnalysisException -cannot resolve '(INTERVAL '2 days' + CAST(1 AS TINYINT))' due to data type mismatch: differing types in '(INTERVAL '2 days' + CAST(1 AS TINYINT))' (interval and tinyint).; line 1 pos 7 +cannot resolve 'CAST(1 AS TINYINT) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS TINYINT)' is of tinyint type.; line 1 pos 7 -- !query 15 @@ -138,7 +138,7 @@ select interval 2 day + cast(1 as smallint) struct<> -- !query 15 output org.apache.spark.sql.AnalysisException -cannot resolve '(INTERVAL '2 days' + CAST(1 AS SMALLINT))' due to data type mismatch: differing types in '(INTERVAL '2 days' + CAST(1 AS SMALLINT))' (interval and smallint).; line 1 pos 7 +cannot resolve 'CAST(1 AS SMALLINT) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS SMALLINT)' is of smallint type.; line 1 pos 7 -- !query 16 @@ -147,7 +147,7 @@ select interval 2 day + cast(1 as int) struct<> -- !query 16 output org.apache.spark.sql.AnalysisException -cannot resolve '(INTERVAL '2 days' + CAST(1 AS INT))' due to data type mismatch: differing types in '(INTERVAL '2 days' + CAST(1 AS INT))' (interval and int).; line 1 pos 7 +cannot resolve 'CAST(1 AS INT) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS INT)' is of int type.; line 1 pos 7 -- !query 17 @@ -156,7 +156,7 @@ select interval 2 day + cast(1 as bigint) struct<> -- !query 17 output org.apache.spark.sql.AnalysisException -cannot resolve '(INTERVAL '2 days' + CAST(1 AS BIGINT))' due to data type mismatch: differing types in '(INTERVAL '2 days' + CAST(1 AS BIGINT))' (interval and bigint).; line 1 pos 7 +cannot resolve 'CAST(1 AS BIGINT) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS BIGINT)' is of bigint type.; line 1 pos 7 -- !query 18 @@ -165,7 +165,7 @@ select interval 2 day + cast(1 as float) struct<> -- !query 18 output org.apache.spark.sql.AnalysisException -cannot resolve '(INTERVAL '2 days' + CAST(1 AS FLOAT))' due to data type mismatch: differing types in '(INTERVAL '2 days' + CAST(1 AS FLOAT))' (interval and float).; line 1 pos 7 +cannot resolve 'CAST(1 AS FLOAT) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS FLOAT)' is of float type.; line 1 pos 7 -- !query 19 @@ -174,7 +174,7 @@ select interval 2 day + cast(1 as double) struct<> -- !query 19 output org.apache.spark.sql.AnalysisException -cannot resolve '(INTERVAL '2 days' + CAST(1 AS DOUBLE))' due to data type mismatch: differing types in '(INTERVAL '2 days' + CAST(1 AS DOUBLE))' (interval and double).; line 1 pos 7 +cannot resolve 'CAST(1 AS DOUBLE) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS DOUBLE)' is of double type.; line 1 pos 7 -- !query 20 @@ -183,7 +183,7 @@ select interval 2 day + cast(1 as decimal(10, 0)) struct<> -- !query 20 output org.apache.spark.sql.AnalysisException -cannot resolve '(INTERVAL '2 days' + CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(INTERVAL '2 days' + CAST(1 AS DECIMAL(10,0)))' (interval and decimal(10,0)).; line 1 pos 7 +cannot resolve 'CAST(1 AS DECIMAL(10,0)) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS DECIMAL(10,0))' is of decimal(10,0) type.; line 1 pos 7 -- !query 21 @@ -208,7 +208,7 @@ select interval 2 day + cast('1' as binary) struct<> -- !query 23 output org.apache.spark.sql.AnalysisException -cannot resolve '(INTERVAL '2 days' + CAST('1' AS BINARY))' due to data type mismatch: differing types in '(INTERVAL '2 days' + CAST('1' AS BINARY))' (interval and binary).; line 1 pos 7 +cannot resolve 'CAST('1' AS BINARY) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST('1' AS BINARY)' is of binary type.; line 1 pos 7 -- !query 24 @@ -217,7 +217,7 @@ select interval 2 day + cast(1 as boolean) struct<> -- !query 24 output org.apache.spark.sql.AnalysisException -cannot resolve '(INTERVAL '2 days' + CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(INTERVAL '2 days' + CAST(1 AS BOOLEAN))' (interval and boolean).; line 1 pos 7 +cannot resolve 'CAST(1 AS BOOLEAN) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS BOOLEAN)' is of boolean type.; line 1 pos 7 -- !query 25 @@ -242,7 +242,7 @@ select cast(1 as tinyint) - interval 2 day struct<> -- !query 27 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS TINYINT) - INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS TINYINT) - INTERVAL '2 days')' (tinyint and interval).; line 1 pos 7 +cannot resolve 'CAST(1 AS TINYINT) - INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS TINYINT)' is of tinyint type.; line 1 pos 7 -- !query 28 @@ -251,7 +251,7 @@ select cast(1 as smallint) - interval 2 day struct<> -- !query 28 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS SMALLINT) - INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS SMALLINT) - INTERVAL '2 days')' (smallint and interval).; line 1 pos 7 +cannot resolve 'CAST(1 AS SMALLINT) - INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS SMALLINT)' is of smallint type.; line 1 pos 7 -- !query 29 @@ -260,7 +260,7 @@ select cast(1 as int) - interval 2 day struct<> -- !query 29 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS INT) - INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS INT) - INTERVAL '2 days')' (int and interval).; line 1 pos 7 +cannot resolve 'CAST(1 AS INT) - INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS INT)' is of int type.; line 1 pos 7 -- !query 30 @@ -269,7 +269,7 @@ select cast(1 as bigint) - interval 2 day struct<> -- !query 30 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS BIGINT) - INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS BIGINT) - INTERVAL '2 days')' (bigint and interval).; line 1 pos 7 +cannot resolve 'CAST(1 AS BIGINT) - INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS BIGINT)' is of bigint type.; line 1 pos 7 -- !query 31 @@ -278,7 +278,7 @@ select cast(1 as float) - interval 2 day struct<> -- !query 31 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS FLOAT) - INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS FLOAT) - INTERVAL '2 days')' (float and interval).; line 1 pos 7 +cannot resolve 'CAST(1 AS FLOAT) - INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS FLOAT)' is of float type.; line 1 pos 7 -- !query 32 @@ -287,7 +287,7 @@ select cast(1 as double) - interval 2 day struct<> -- !query 32 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DOUBLE) - INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS DOUBLE) - INTERVAL '2 days')' (double and interval).; line 1 pos 7 +cannot resolve 'CAST(1 AS DOUBLE) - INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS DOUBLE)' is of double type.; line 1 pos 7 -- !query 33 @@ -296,7 +296,7 @@ select cast(1 as decimal(10, 0)) - interval 2 day struct<> -- !query 33 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DECIMAL(10,0)) - INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) - INTERVAL '2 days')' (decimal(10,0) and interval).; line 1 pos 7 +cannot resolve 'CAST(1 AS DECIMAL(10,0)) - INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS DECIMAL(10,0))' is of decimal(10,0) type.; line 1 pos 7 -- !query 34 @@ -321,7 +321,7 @@ select cast('1' as binary) - interval 2 day struct<> -- !query 36 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('1' AS BINARY) - INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST('1' AS BINARY) - INTERVAL '2 days')' (binary and interval).; line 1 pos 7 +cannot resolve 'CAST('1' AS BINARY) - INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST('1' AS BINARY)' is of binary type.; line 1 pos 7 -- !query 37 @@ -330,7 +330,7 @@ select cast(1 as boolean) - interval 2 day struct<> -- !query 37 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS BOOLEAN) - INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) - INTERVAL '2 days')' (boolean and interval).; line 1 pos 7 +cannot resolve 'CAST(1 AS BOOLEAN) - INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS BOOLEAN)' is of boolean type.; line 1 pos 7 -- !query 38 diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out index e479a6d73a18..ec6f3d7696b0 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out @@ -312,7 +312,7 @@ SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(3, 0)) FROM t struct<> -- !query 37 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) + CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) + CAST(1 AS DECIMAL(3,0)))' (date and decimal(3,0)).; line 1 pos 7 +cannot resolve 'date_add(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(1 AS DECIMAL(3,0))' is of decimal(3,0) type.; line 1 pos 7 -- !query 38 @@ -321,7 +321,7 @@ SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(5, 0)) FROM t struct<> -- !query 38 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) + CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) + CAST(1 AS DECIMAL(5,0)))' (date and decimal(5,0)).; line 1 pos 7 +cannot resolve 'date_add(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(1 AS DECIMAL(5,0))' is of decimal(5,0) type.; line 1 pos 7 -- !query 39 @@ -330,7 +330,7 @@ SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(10, 0)) FROM t struct<> -- !query 39 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) + CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) + CAST(1 AS DECIMAL(10,0)))' (date and decimal(10,0)).; line 1 pos 7 +cannot resolve 'date_add(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(1 AS DECIMAL(10,0))' is of decimal(10,0) type.; line 1 pos 7 -- !query 40 @@ -339,7 +339,7 @@ SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(20, 0)) FROM t struct<> -- !query 40 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) + CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) + CAST(1 AS DECIMAL(20,0)))' (date and decimal(20,0)).; line 1 pos 7 +cannot resolve 'date_add(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(1 AS DECIMAL(20,0))' is of decimal(20,0) type.; line 1 pos 7 -- !query 41 @@ -712,7 +712,7 @@ SELECT cast(1 as decimal(3, 0)) + cast('2017-12-11 09:30:00' as date) FROM t struct<> -- !query 85 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DECIMAL(3,0)) + CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) + CAST('2017-12-11 09:30:00' AS DATE))' (decimal(3,0) and date).; line 1 pos 7 +cannot resolve 'date_add(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(1 AS DECIMAL(3,0))' is of decimal(3,0) type.; line 1 pos 7 -- !query 86 @@ -721,7 +721,7 @@ SELECT cast(1 as decimal(5, 0)) + cast('2017-12-11 09:30:00' as date) FROM t struct<> -- !query 86 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DECIMAL(5,0)) + CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) + CAST('2017-12-11 09:30:00' AS DATE))' (decimal(5,0) and date).; line 1 pos 7 +cannot resolve 'date_add(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(1 AS DECIMAL(5,0))' is of decimal(5,0) type.; line 1 pos 7 -- !query 87 @@ -730,7 +730,7 @@ SELECT cast(1 as decimal(10, 0)) + cast('2017-12-11 09:30:00' as date) FROM t struct<> -- !query 87 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DECIMAL(10,0)) + CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) + CAST('2017-12-11 09:30:00' AS DATE))' (decimal(10,0) and date).; line 1 pos 7 +cannot resolve 'date_add(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(1 AS DECIMAL(10,0))' is of decimal(10,0) type.; line 1 pos 7 -- !query 88 @@ -739,7 +739,7 @@ SELECT cast(1 as decimal(20, 0)) + cast('2017-12-11 09:30:00' as date) FROM t struct<> -- !query 88 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DECIMAL(20,0)) + CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) + CAST('2017-12-11 09:30:00' AS DATE))' (decimal(20,0) and date).; line 1 pos 7 +cannot resolve 'date_add(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(1 AS DECIMAL(20,0))' is of decimal(20,0) type.; line 1 pos 7 -- !query 89 @@ -1008,7 +1008,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(3, 0)) FRO struct<> -- !query 121 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) - CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) - CAST(1 AS DECIMAL(3,0)))' (timestamp and decimal(3,0)).; line 1 pos 7 +cannot resolve 'subtracttimestamps(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP), CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: argument 2 requires timestamp type, however, 'CAST(1 AS DECIMAL(3,0))' is of decimal(3,0) type.; line 1 pos 7 -- !query 122 @@ -1017,7 +1017,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(5, 0)) FRO struct<> -- !query 122 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) - CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) - CAST(1 AS DECIMAL(5,0)))' (timestamp and decimal(5,0)).; line 1 pos 7 +cannot resolve 'subtracttimestamps(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP), CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: argument 2 requires timestamp type, however, 'CAST(1 AS DECIMAL(5,0))' is of decimal(5,0) type.; line 1 pos 7 -- !query 123 @@ -1026,7 +1026,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(10, 0)) FR struct<> -- !query 123 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) - CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) - CAST(1 AS DECIMAL(10,0)))' (timestamp and decimal(10,0)).; line 1 pos 7 +cannot resolve 'subtracttimestamps(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP), CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: argument 2 requires timestamp type, however, 'CAST(1 AS DECIMAL(10,0))' is of decimal(10,0) type.; line 1 pos 7 -- !query 124 @@ -1035,7 +1035,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(20, 0)) FR struct<> -- !query 124 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) - CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) - CAST(1 AS DECIMAL(20,0)))' (timestamp and decimal(20,0)).; line 1 pos 7 +cannot resolve 'subtracttimestamps(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP), CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: argument 2 requires timestamp type, however, 'CAST(1 AS DECIMAL(20,0))' is of decimal(20,0) type.; line 1 pos 7 -- !query 125 @@ -1044,7 +1044,7 @@ SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(3, 0)) FROM t struct<> -- !query 125 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) - CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) - CAST(1 AS DECIMAL(3,0)))' (date and decimal(3,0)).; line 1 pos 7 +cannot resolve 'date_sub(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(1 AS DECIMAL(3,0))' is of decimal(3,0) type.; line 1 pos 7 -- !query 126 @@ -1053,7 +1053,7 @@ SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(5, 0)) FROM t struct<> -- !query 126 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) - CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) - CAST(1 AS DECIMAL(5,0)))' (date and decimal(5,0)).; line 1 pos 7 +cannot resolve 'date_sub(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(1 AS DECIMAL(5,0))' is of decimal(5,0) type.; line 1 pos 7 -- !query 127 @@ -1062,7 +1062,7 @@ SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(10, 0)) FROM t struct<> -- !query 127 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) - CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) - CAST(1 AS DECIMAL(10,0)))' (date and decimal(10,0)).; line 1 pos 7 +cannot resolve 'date_sub(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(1 AS DECIMAL(10,0))' is of decimal(10,0) type.; line 1 pos 7 -- !query 128 @@ -1071,7 +1071,7 @@ SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(20, 0)) FROM t struct<> -- !query 128 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) - CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) - CAST(1 AS DECIMAL(20,0)))' (date and decimal(20,0)).; line 1 pos 7 +cannot resolve 'date_sub(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(1 AS DECIMAL(20,0))' is of decimal(20,0) type.; line 1 pos 7 -- !query 129 @@ -1408,7 +1408,7 @@ SELECT cast(1 as decimal(3, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FR struct<> -- !query 169 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DECIMAL(3,0)) - CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) - CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(3,0) and timestamp).; line 1 pos 7 +cannot resolve 'subtracttimestamps(CAST(1 AS DECIMAL(3,0)), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS DECIMAL(3,0))' is of decimal(3,0) type.; line 1 pos 7 -- !query 170 @@ -1417,7 +1417,7 @@ SELECT cast(1 as decimal(5, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FR struct<> -- !query 170 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DECIMAL(5,0)) - CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) - CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(5,0) and timestamp).; line 1 pos 7 +cannot resolve 'subtracttimestamps(CAST(1 AS DECIMAL(5,0)), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS DECIMAL(5,0))' is of decimal(5,0) type.; line 1 pos 7 -- !query 171 @@ -1426,7 +1426,7 @@ SELECT cast(1 as decimal(10, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FR struct<> -- !query 171 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DECIMAL(10,0)) - CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) - CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(10,0) and timestamp).; line 1 pos 7 +cannot resolve 'subtracttimestamps(CAST(1 AS DECIMAL(10,0)), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS DECIMAL(10,0))' is of decimal(10,0) type.; line 1 pos 7 -- !query 172 @@ -1435,7 +1435,7 @@ SELECT cast(1 as decimal(20, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FR struct<> -- !query 172 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DECIMAL(20,0)) - CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) - CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(20,0) and timestamp).; line 1 pos 7 +cannot resolve 'subtracttimestamps(CAST(1 AS DECIMAL(20,0)), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS DECIMAL(20,0))' is of decimal(20,0) type.; line 1 pos 7 -- !query 173 @@ -1444,7 +1444,7 @@ SELECT cast(1 as decimal(3, 0)) - cast('2017-12-11 09:30:00' as date) FROM t struct<> -- !query 173 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DECIMAL(3,0)) - CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) - CAST('2017-12-11 09:30:00' AS DATE))' (decimal(3,0) and date).; line 1 pos 7 +cannot resolve 'subtractdates(CAST(1 AS DECIMAL(3,0)), CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: argument 1 requires date type, however, 'CAST(1 AS DECIMAL(3,0))' is of decimal(3,0) type.; line 1 pos 7 -- !query 174 @@ -1453,7 +1453,7 @@ SELECT cast(1 as decimal(5, 0)) - cast('2017-12-11 09:30:00' as date) FROM t struct<> -- !query 174 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DECIMAL(5,0)) - CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) - CAST('2017-12-11 09:30:00' AS DATE))' (decimal(5,0) and date).; line 1 pos 7 +cannot resolve 'subtractdates(CAST(1 AS DECIMAL(5,0)), CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: argument 1 requires date type, however, 'CAST(1 AS DECIMAL(5,0))' is of decimal(5,0) type.; line 1 pos 7 -- !query 175 @@ -1462,7 +1462,7 @@ SELECT cast(1 as decimal(10, 0)) - cast('2017-12-11 09:30:00' as date) FROM t struct<> -- !query 175 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DECIMAL(10,0)) - CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) - CAST('2017-12-11 09:30:00' AS DATE))' (decimal(10,0) and date).; line 1 pos 7 +cannot resolve 'subtractdates(CAST(1 AS DECIMAL(10,0)), CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: argument 1 requires date type, however, 'CAST(1 AS DECIMAL(10,0))' is of decimal(10,0) type.; line 1 pos 7 -- !query 176 @@ -1471,7 +1471,7 @@ SELECT cast(1 as decimal(20, 0)) - cast('2017-12-11 09:30:00' as date) FROM t struct<> -- !query 176 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DECIMAL(20,0)) - CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) - CAST('2017-12-11 09:30:00' AS DATE))' (decimal(20,0) and date).; line 1 pos 7 +cannot resolve 'subtractdates(CAST(1 AS DECIMAL(20,0)), CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: argument 1 requires date type, however, 'CAST(1 AS DECIMAL(20,0))' is of decimal(20,0) type.; line 1 pos 7 -- !query 177 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 c54ceba85ce7..4e7aa517b62c 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 @@ -107,7 +107,7 @@ SELECT '1' + cast('2017-12-11 09:30:00' as date) FROM t struct<> -- !query 12 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('1' AS DOUBLE) + CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST('1' AS DOUBLE) + CAST('2017-12-11 09:30:00' AS DATE))' (double and date).; line 1 pos 7 +cannot resolve 'date_add(CAST('2017-12-11 09:30:00' AS DATE), '1')' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, ''1'' is of string type.; line 1 pos 7 -- !query 13 @@ -198,16 +198,15 @@ SELECT '1' - cast('2017-12-11 09:30:00.0' as timestamp) FROM t struct<> -- !query 23 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('1' AS DOUBLE) - CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST('1' AS DOUBLE) - CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (double and timestamp).; line 1 pos 7 +cannot resolve 'subtracttimestamps('1', CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: argument 1 requires timestamp type, however, ''1'' is of string type.; line 1 pos 7 -- !query 24 SELECT '1' - cast('2017-12-11 09:30:00' as date) FROM t -- !query 24 schema -struct<> +struct -- !query 24 output -org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('1' AS DOUBLE) - CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST('1' AS DOUBLE) - CAST('2017-12-11 09:30:00' AS DATE))' (double and date).; line 1 pos 7 +NULL -- !query 25 @@ -699,7 +698,7 @@ SELECT cast('2017-12-11 09:30:00' as date) + '1' FROM t struct<> -- !query 83 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) + CAST('1' AS DOUBLE))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) + CAST('1' AS DOUBLE))' (date and double).; line 1 pos 7 +cannot resolve 'date_add(CAST('2017-12-11 09:30:00' AS DATE), '1')' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, ''1'' is of string type.; line 1 pos 7 -- !query 84 @@ -782,7 +781,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) - '1' FROM t struct<> -- !query 93 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) - CAST('1' AS DOUBLE))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) - CAST('1' AS DOUBLE))' (timestamp and double).; line 1 pos 7 +cannot resolve 'subtracttimestamps(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP), '1')' due to data type mismatch: argument 2 requires timestamp type, however, ''1'' is of string type.; line 1 pos 7 -- !query 94 @@ -791,7 +790,7 @@ SELECT cast('2017-12-11 09:30:00' as date) - '1' FROM t struct<> -- !query 94 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) - CAST('1' AS DOUBLE))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) - CAST('1' AS DOUBLE))' (date and double).; line 1 pos 7 +cannot resolve 'date_sub(CAST('2017-12-11 09:30:00' AS DATE), '1')' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, ''1'' is of string type.; line 1 pos 7 -- !query 95