Skip to content
Closed
Show file tree
Hide file tree
Changes from all 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 @@ -412,12 +412,17 @@ object TypeCoercion {
case e if !e.childrenResolved => e

case a @ BinaryArithmetic(left @ StringType(), right)
if right.dataType != CalendarIntervalType =>
if right.dataType != CalendarIntervalType && right.dataType != TimestampType =>
a.makeCopy(Array(Cast(left, DoubleType), right))
case a @ BinaryArithmetic(left, right @ StringType())
if left.dataType != CalendarIntervalType =>
if left.dataType != CalendarIntervalType && left.dataType != TimestampType =>
a.makeCopy(Array(left, Cast(right, DoubleType)))

case a @ BinaryArithmetic(left @ TimestampType(), right @ StringType()) =>
a.makeCopy(Array(left, Cast(right, CalendarIntervalType)))
case a @ BinaryArithmetic(left @ StringType(), right @ TimestampType()) =>
a.makeCopy(Array(Cast(left, CalendarIntervalType), right))

// For equality between string and timestamp we cast the string to a timestamp
// so that things like rounding of subsecond precision does not affect the comparison.
case p @ Equality(left @ StringType(), right @ TimestampType()) =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1576,6 +1576,21 @@ class TypeCoercionSuite extends AnalysisTest {
SpecifiedWindowFrame(RangeFrame, CurrentRow, UnboundedFollowing))
)
}

test("binary arithmetic with string promotion") {
val rule = TypeCoercion.PromoteStrings(conf)
val timestamp = Literal(new Timestamp(0L))
val intervalStr = Literal("interval 3 month 7 hours")
ruleTest(rule,
Add(timestamp, intervalStr),
Add(timestamp, Cast(intervalStr, CalendarIntervalType)))
ruleTest(rule,
Add(intervalStr, timestamp),
Add(Cast(intervalStr, CalendarIntervalType), timestamp))
ruleTest(rule,
Subtract(timestamp, intervalStr),
Subtract(timestamp, Cast(intervalStr, CalendarIntervalType)))
}
}


Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -798,7 +798,7 @@ SELECT cast(1 as string) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t
struct<>
-- !query 95 output
org.apache.spark.sql.AnalysisException
cannot resolve '(CAST(CAST(1 AS STRING) AS DOUBLE) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(CAST(1 AS STRING) AS DOUBLE) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (double and timestamp).; line 1 pos 7
cannot resolve '(CAST(CAST(1 AS STRING) AS INTERVAL) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(CAST(1 AS STRING) AS INTERVAL) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (interval and timestamp).; line 1 pos 7


-- !query 96
Expand Down Expand Up @@ -1095,7 +1095,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as string) FROM t
struct<>
-- !query 128 output
org.apache.spark.sql.AnalysisException
cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(CAST(1 AS STRING) AS DOUBLE))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(CAST(1 AS STRING) AS DOUBLE))' (timestamp and double).; line 1 pos 7
cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(CAST(1 AS STRING) AS INTERVAL))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(CAST(1 AS STRING) AS INTERVAL))' (timestamp and interval).; line 1 pos 7


-- !query 129
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -95,10 +95,9 @@ cannot resolve '(CAST('1' AS DOUBLE) + CAST(1 AS BOOLEAN))' due to data type mis
-- !query 11
SELECT '1' + cast('2017-12-11 09:30:00.0' as timestamp) FROM t
-- !query 11 schema
struct<>
struct<CAST(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) + CAST(1 AS INTERVAL) AS TIMESTAMP):timestamp>
-- !query 11 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
NULL


-- !query 12
Expand Down Expand Up @@ -198,7 +197,7 @@ 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 '(CAST('1' AS INTERVAL) - CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST('1' AS INTERVAL) - CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (interval and timestamp).; line 1 pos 7


-- !query 24
Expand Down Expand Up @@ -298,7 +297,7 @@ SELECT '1' * cast('2017-12-11 09:30:00.0' as timestamp) FROM t
struct<>
-- !query 35 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 '(CAST('1' AS INTERVAL) * CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST('1' AS INTERVAL) * CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (interval and timestamp).; line 1 pos 7


-- !query 36
Expand Down Expand Up @@ -398,7 +397,7 @@ SELECT '1' / cast('2017-12-11 09:30:00.0' as timestamp) FROM t
struct<>
-- !query 47 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 '(CAST('1' AS INTERVAL) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST('1' AS INTERVAL) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (interval and timestamp).; line 1 pos 7


-- !query 48
Expand Down Expand Up @@ -498,7 +497,7 @@ SELECT '1' % cast('2017-12-11 09:30:00.0' as timestamp) FROM t
struct<>
-- !query 59 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 '(CAST('1' AS INTERVAL) % CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST('1' AS INTERVAL) % CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (interval and timestamp).; line 1 pos 7


-- !query 60
Expand Down Expand Up @@ -598,7 +597,7 @@ SELECT pmod('1', cast('2017-12-11 09:30:00.0' as timestamp)) FROM t
struct<>
-- !query 71 output
org.apache.spark.sql.AnalysisException
cannot resolve 'pmod(CAST('1' AS DOUBLE), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in 'pmod(CAST('1' AS DOUBLE), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (double and timestamp).; line 1 pos 7
cannot resolve 'pmod(CAST('1' AS INTERVAL), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in 'pmod(CAST('1' AS INTERVAL), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (interval and timestamp).; line 1 pos 7


-- !query 72
Expand Down Expand Up @@ -687,10 +686,9 @@ cannot resolve '(CAST(1 AS BOOLEAN) + CAST('1' AS DOUBLE))' due to data type mis
-- !query 82
SELECT cast('2017-12-11 09:30:00.0' as timestamp) + '1' FROM t
-- !query 82 schema
struct<>
struct<CAST(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) + CAST(1 AS INTERVAL) AS TIMESTAMP):timestamp>
-- !query 82 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
NULL


-- !query 83
Expand Down Expand Up @@ -779,10 +777,9 @@ cannot resolve '(CAST(1 AS BOOLEAN) - CAST('1' AS DOUBLE))' due to data type mis
-- !query 93
SELECT cast('2017-12-11 09:30:00.0' as timestamp) - '1' FROM t
-- !query 93 schema
struct<>
struct<CAST(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) - CAST(1 AS INTERVAL) AS TIMESTAMP):timestamp>
-- !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
NULL


-- !query 94
Expand Down Expand Up @@ -874,7 +871,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) * '1' FROM t
struct<>
-- !query 104 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 '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) * CAST('1' AS INTERVAL))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) * CAST('1' AS INTERVAL))' (timestamp and interval).; line 1 pos 7


-- !query 105
Expand Down Expand Up @@ -966,7 +963,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) / '1' FROM t
struct<>
-- !query 115 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 '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST('1' AS INTERVAL))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST('1' AS INTERVAL))' (timestamp and interval).; line 1 pos 7


-- !query 116
Expand Down Expand Up @@ -1058,7 +1055,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) % '1' FROM t
struct<>
-- !query 126 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 '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) % CAST('1' AS INTERVAL))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) % CAST('1' AS INTERVAL))' (timestamp and interval).; line 1 pos 7


-- !query 127
Expand Down Expand Up @@ -1150,7 +1147,7 @@ SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), '1') FROM t
struct<>
-- !query 137 output
org.apache.spark.sql.AnalysisException
cannot resolve 'pmod(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP), CAST('1' AS DOUBLE))' due to data type mismatch: differing types in 'pmod(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP), CAST('1' AS DOUBLE))' (timestamp and double).; line 1 pos 7
cannot resolve 'pmod(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP), CAST('1' AS INTERVAL))' due to data type mismatch: differing types in 'pmod(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP), CAST('1' AS INTERVAL))' (timestamp and interval).; line 1 pos 7


-- !query 138
Expand Down