From f59a213027968f5fe157376db51d0e15b68b47db Mon Sep 17 00:00:00 2001
From: Yuming Wang
Date: Sat, 5 Aug 2017 17:20:17 +0800
Subject: [PATCH 01/19] BinaryComparison shouldn't auto cast string to int/long
---
.../spark/sql/catalyst/analysis/TypeCoercion.scala | 8 ++++++++
.../test/scala/org/apache/spark/sql/DataFrameSuite.scala | 9 +++++++++
2 files changed, 17 insertions(+)
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 25af014f67fe9..1b7fc6cd4e928 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
@@ -353,6 +353,14 @@ object TypeCoercion {
case p @ Equality(left @ TimestampType(), right @ StringType()) =>
p.makeCopy(Array(left, Cast(right, TimestampType)))
+ case p @ BinaryComparison(left, right)
+ if left.isInstanceOf[AttributeReference] && right.isInstanceOf[Literal] =>
+ p.makeCopy(Array(left, castExpr(right, left.dataType)))
+
+ case p @ BinaryComparison(left, right)
+ if left.isInstanceOf[Literal] && right.isInstanceOf[AttributeReference] =>
+ p.makeCopy(Array(castExpr(left, right.dataType), right))
+
case p @ BinaryComparison(left, right)
if findCommonTypeForBinaryComparison(left.dataType, right.dataType).isDefined =>
val commonType = findCommonTypeForBinaryComparison(left.dataType, right.dataType).get
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
index 7c500728bdec9..2e175ca534072 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
@@ -1752,6 +1752,15 @@ class DataFrameSuite extends QueryTest with SharedSQLContext {
Row(1, "a"))
}
+ test("SPARK-21646: BinaryComparison shouldn't auto cast string to int/long") {
+ val str1 = Long.MaxValue.toString + "1"
+ val str2 = Int.MaxValue.toString + "1"
+ val str3 = "10"
+ val df = Seq(str1 -> "a", str2 -> "b", str3 -> "c").toDF("c1", "c2")
+ assert(df.filter($"c1" > 0).count() === 3)
+ assert(df.filter($"c1" > 0L).count() === 3)
+ }
+
test("SPARK-12982: Add table name validation in temp table registration") {
val df = Seq("foo", "bar").map(Tuple1.apply).toDF("col")
// invalid table names
From bc83848cb797861addf5a89c2ae0a456e169ccd1 Mon Sep 17 00:00:00 2001
From: Yuming Wang
Date: Sun, 10 Sep 2017 23:16:50 +0800
Subject: [PATCH 02/19] follow hive
---
.../sql/catalyst/analysis/TypeCoercion.scala | 36 +++++-----
.../catalyst/analysis/TypeCoercionSuite.scala | 25 ++++++-
.../org/apache/spark/sql/DataFrameSuite.scala | 9 ---
.../org/apache/spark/sql/SQLQuerySuite.scala | 65 ++++++++++++++++++-
4 files changed, 106 insertions(+), 29 deletions(-)
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 1b7fc6cd4e928..0dd4f71c91d11 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
@@ -116,19 +116,21 @@ object TypeCoercion {
* other is a Timestamp by making the target type to be String.
*/
val findCommonTypeForBinaryComparison: (DataType, DataType) => Option[DataType] = {
- // We should cast all relative timestamp/date/string comparison into string comparisons
- // This behaves as a user would expect because timestamp strings sort lexicographically.
- // i.e. TimeStamp(2013-01-01 00:00 ...) < "2014" = true
- case (StringType, DateType) => Some(StringType)
- case (DateType, StringType) => Some(StringType)
- case (StringType, TimestampType) => Some(StringType)
- case (TimestampType, StringType) => Some(StringType)
- case (TimestampType, DateType) => Some(StringType)
- case (DateType, TimestampType) => Some(StringType)
+ // We should follow hive:
+ // https://github.com/apache/hive/blob/rel/storage-release-2.4.0/ql/src/java/
+ // org/apache/hadoop/hive/ql/exec/FunctionRegistry.java#L781
+ case (StringType, DateType) => Some(DateType)
+ case (DateType, StringType) => Some(DateType)
+ case (StringType, TimestampType) => Some(TimestampType)
+ case (TimestampType, StringType) => Some(TimestampType)
+ case (TimestampType, DateType) => Some(TimestampType)
+ case (DateType, TimestampType) => Some(TimestampType)
case (StringType, NullType) => Some(StringType)
case (NullType, StringType) => Some(StringType)
+ case (StringType | TimestampType, r: NumericType) => Some(DoubleType)
+ case (l: NumericType, StringType | TimestampType) => Some(DoubleType)
case (l: StringType, r: AtomicType) if r != StringType => Some(r)
- case (l: AtomicType, r: StringType) if (l != StringType) => Some(l)
+ case (l: AtomicType, r: StringType) if l != StringType => Some(l)
case (l, r) => None
}
@@ -353,13 +355,13 @@ object TypeCoercion {
case p @ Equality(left @ TimestampType(), right @ StringType()) =>
p.makeCopy(Array(left, Cast(right, TimestampType)))
- case p @ BinaryComparison(left, right)
- if left.isInstanceOf[AttributeReference] && right.isInstanceOf[Literal] =>
- p.makeCopy(Array(left, castExpr(right, left.dataType)))
-
- case p @ BinaryComparison(left, right)
- if left.isInstanceOf[Literal] && right.isInstanceOf[AttributeReference] =>
- p.makeCopy(Array(castExpr(left, right.dataType), right))
+// case p @ BinaryComparison(left, right)
+// if left.isInstanceOf[AttributeReference] && right.isInstanceOf[Literal] =>
+// p.makeCopy(Array(left, castExpr(right, left.dataType)))
+//
+// case p @ BinaryComparison(left, right)
+// if left.isInstanceOf[Literal] && right.isInstanceOf[AttributeReference] =>
+// p.makeCopy(Array(castExpr(left, right.dataType), right))
case p @ BinaryComparison(left, right)
if findCommonTypeForBinaryComparison(left.dataType, right.dataType).isDefined =>
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 d62e3b6dfe34f..d93a3d957fbe5 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
@@ -17,7 +17,7 @@
package org.apache.spark.sql.catalyst.analysis
-import java.sql.Timestamp
+import java.sql.{Date, Timestamp}
import org.apache.spark.sql.catalyst.analysis.TypeCoercion._
import org.apache.spark.sql.catalyst.dsl.expressions._
@@ -1101,13 +1101,34 @@ class TypeCoercionSuite extends AnalysisTest {
test("binary comparison with string promotion") {
ruleTest(PromoteStrings,
GreaterThan(Literal("123"), Literal(1)),
- GreaterThan(Cast(Literal("123"), IntegerType), Literal(1)))
+ GreaterThan(Cast(Literal("123"), DoubleType), Cast(Literal(1), DoubleType)))
ruleTest(PromoteStrings,
LessThan(Literal(true), Literal("123")),
LessThan(Literal(true), Cast(Literal("123"), BooleanType)))
ruleTest(PromoteStrings,
EqualTo(Literal(Array(1, 2)), Literal("123")),
EqualTo(Literal(Array(1, 2)), Literal("123")))
+ ruleTest(PromoteStrings,
+ GreaterThan(Literal("123"), Literal(1L)),
+ GreaterThan(Cast(Literal("123"), DoubleType), Cast(Literal(1L), DoubleType)))
+ ruleTest(PromoteStrings,
+ GreaterThan(Literal("123"), Literal(0.1)),
+ GreaterThan(Cast(Literal("123"), DoubleType), Literal(0.1)))
+
+ val date1 = Date.valueOf("2017-07-21")
+ val timestamp1 = Timestamp.valueOf("2017-07-21 23:42:12.123")
+ ruleTest(PromoteStrings,
+ GreaterThan(Literal(date1), Literal("2017-07-01")),
+ GreaterThan(Literal(date1), Cast(Literal("2017-07-01"), DateType)))
+ ruleTest(PromoteStrings,
+ GreaterThan(Literal(timestamp1), Literal("2017-07-01")),
+ GreaterThan(Literal(timestamp1), Cast(Literal("2017-07-01"), TimestampType)))
+ ruleTest(PromoteStrings,
+ GreaterThan(Literal(timestamp1), Cast(Literal("2017-07-01"), DateType)),
+ GreaterThan(Literal(timestamp1), Cast(Cast(Literal("2017-07-01"), DateType), TimestampType)))
+ ruleTest(PromoteStrings,
+ GreaterThan(Literal(timestamp1), Literal(1L)),
+ GreaterThan(Cast(Literal(timestamp1), DoubleType), Cast(Literal(1L), DoubleType)))
}
test("cast WindowFrame boundaries to the type they operate upon") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
index 2e175ca534072..7c500728bdec9 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
@@ -1752,15 +1752,6 @@ class DataFrameSuite extends QueryTest with SharedSQLContext {
Row(1, "a"))
}
- test("SPARK-21646: BinaryComparison shouldn't auto cast string to int/long") {
- val str1 = Long.MaxValue.toString + "1"
- val str2 = Int.MaxValue.toString + "1"
- val str3 = "10"
- val df = Seq(str1 -> "a", str2 -> "b", str3 -> "c").toDF("c1", "c2")
- assert(df.filter($"c1" > 0).count() === 3)
- assert(df.filter($"c1" > 0L).count() === 3)
- }
-
test("SPARK-12982: Add table name validation in temp table registration") {
val df = Seq("foo", "bar").map(Tuple1.apply).toDF("col")
// invalid table names
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
index e95f6dba46079..4264778625891 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql
import java.io.File
import java.math.MathContext
import java.net.{MalformedURLException, URL}
-import java.sql.Timestamp
+import java.sql.{Date, Timestamp}
import java.util.concurrent.atomic.AtomicBoolean
import org.apache.spark.{AccumulatorSuite, SparkException}
@@ -2658,4 +2658,67 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
checkAnswer(sql("SELECT __auto_generated_subquery_name.i from (SELECT i FROM v)"), Row(1))
}
}
+
+ test("SPARK-21646: CommonTypeForBinaryComparison: StringType vs NumericType") {
+ withTempView("v") {
+ val str1 = Long.MaxValue.toString + "1"
+ val str2 = Int.MaxValue.toString + "1"
+ val str3 = "10"
+ Seq(str1, str2, str3).toDF("c1").createOrReplaceTempView("v")
+ checkAnswer(sql("SELECT c1 from v where c1 > 0"), Row(str1) :: Row(str2) :: Row(str3) :: Nil)
+ checkAnswer(sql("SELECT c1 from v where c1 > 0L"), Row(str1) :: Row(str2) :: Row(str3) :: Nil)
+ }
+ }
+
+ test("SPARK-21646: CommonTypeForBinaryComparison: DoubleType vs IntegerType") {
+ withTempView("v") {
+ Seq(("0", 1), ("-0.4", 2)).toDF("a", "b").createOrReplaceTempView("v")
+ checkAnswer(sql("SELECT a FROM v WHERE a=0"), Seq(Row("0")))
+ checkAnswer(sql("SELECT a FROM v WHERE a=0L"), Seq(Row("0")))
+ checkAnswer(sql("SELECT a FROM v WHERE a=0.0"), Seq(Row("0")))
+ checkAnswer(sql("SELECT a FROM v WHERE a=-0.4"), Seq(Row("-0.4")))
+ }
+ }
+
+ test("SPARK-21646: CommonTypeForBinaryComparison: StringType vs DateType") {
+ withTempView("v") {
+ val v1 = Date.valueOf("2017-09-22")
+ val v2 = Date.valueOf("2017-09-09")
+ Seq(v1, v2).toDF("c1").createTempView("v")
+ checkAnswer(sql("select * from v where c1 > '2017-8-1'"), Row(v1) :: Row(v2) :: Nil)
+ checkAnswer(sql("select * from v where c1 > cast('2017-8-1' as date)"),
+ Row(v1) :: Row(v2) :: Nil)
+ }
+ }
+
+ test("SPARK-21646: CommonTypeForBinaryComparison: StringType vs TimestampType") {
+ withTempView("v") {
+ val v1 = Timestamp.valueOf("2017-07-21 23:42:12.123")
+ val v2 = Timestamp.valueOf("2017-08-21 23:42:12.123")
+ val df = Seq(v1, v2).toDF("c1").createTempView("v")
+ checkAnswer(sql("select * from v where c1 > '2017-8-1'"), Row(v2) :: Nil)
+ checkAnswer(sql("select * from v where c1 > cast('2017-8-1' as timestamp)"), Row(v2) :: Nil)
+ }
+ }
+
+ test("SPARK-21646: CommonTypeForBinaryComparison: TimestampType vs DateType") {
+ withTempView("v") {
+ val v1 = Timestamp.valueOf("2017-07-21 23:42:12.123")
+ val v2 = Timestamp.valueOf("2017-08-21 23:42:12.123")
+ val df = Seq(v1, v2).toDF("c1").createTempView("v")
+ checkAnswer(sql("select * from v where c1 > cast('2017-8-1' as date)"), Row(v2) :: Nil)
+ checkAnswer(sql("select * from v where c1 > cast('2017-8-1' as timestamp)"), Row(v2) :: Nil)
+ }
+ }
+
+ test("SPARK-21646: CommonTypeForBinaryComparison: TimestampType vs NumericType") {
+ withTempView("v") {
+ val v1 = Timestamp.valueOf("2017-07-21 23:42:12.123")
+ val v2 = Timestamp.valueOf("2017-08-21 23:42:12.123")
+ val df = Seq(v1, v2).toDF("c1").createTempView("v")
+ checkAnswer(sql("select * from v where c1 > 1"), Row(v1) :: Row(v2) :: Nil)
+ checkAnswer(sql("select * from v where c1 > cast(cast('2010-08-01' as timestamp) as double)"),
+ Row(v1) :: Row(v2) :: Nil)
+ }
+ }
}
From cedb2397dcd3f68d313fb447675d923eea628ebf Mon Sep 17 00:00:00 2001
From: Yuming Wang
Date: Sun, 10 Sep 2017 23:19:26 +0800
Subject: [PATCH 03/19] Remove useless code
---
.../spark/sql/catalyst/analysis/TypeCoercion.scala | 9 ---------
1 file changed, 9 deletions(-)
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 0dd4f71c91d11..603de3debc78e 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
@@ -354,15 +354,6 @@ object TypeCoercion {
p.makeCopy(Array(Cast(left, TimestampType), right))
case p @ Equality(left @ TimestampType(), right @ StringType()) =>
p.makeCopy(Array(left, Cast(right, TimestampType)))
-
-// case p @ BinaryComparison(left, right)
-// if left.isInstanceOf[AttributeReference] && right.isInstanceOf[Literal] =>
-// p.makeCopy(Array(left, castExpr(right, left.dataType)))
-//
-// case p @ BinaryComparison(left, right)
-// if left.isInstanceOf[Literal] && right.isInstanceOf[AttributeReference] =>
-// p.makeCopy(Array(castExpr(left, right.dataType), right))
-
case p @ BinaryComparison(left, right)
if findCommonTypeForBinaryComparison(left.dataType, right.dataType).isDefined =>
val commonType = findCommonTypeForBinaryComparison(left.dataType, right.dataType).get
From 522c4cddbd1ac0634fe5381d8c95df64c1eeb9c6 Mon Sep 17 00:00:00 2001
From: Yuming Wang
Date: Mon, 11 Sep 2017 08:57:47 +0800
Subject: [PATCH 04/19] Fix test error
---
.../src/test/scala/org/apache/spark/sql/DataFrameSuite.scala | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
index 13341645e8ff8..6f01bc40259cb 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
@@ -1966,7 +1966,7 @@ class DataFrameSuite extends QueryTest with SharedSQLContext {
test("SPARK-17913: compare long and string type column may return confusing result") {
val df = Seq(123L -> "123", 19157170390056973L -> "19157170390056971").toDF("i", "j")
- checkAnswer(df.select($"i" === $"j"), Row(true) :: Row(false) :: Nil)
+ checkAnswer(df.select($"i" === $"j"), Row(true) :: Row(true) :: Nil)
}
test("SPARK-19691 Calculating percentile of decimal column fails with ClassCastException") {
From 3bec6a22565c58ad2da18d66e1e285d644f3577a Mon Sep 17 00:00:00 2001
From: Yuming Wang
Date: Mon, 11 Sep 2017 19:52:45 +0800
Subject: [PATCH 05/19] Fix SQLQueryTestSuite test error
---
.../results/predicate-functions.sql.out | 48 +++++++++----------
1 file changed, 24 insertions(+), 24 deletions(-)
diff --git a/sql/core/src/test/resources/sql-tests/results/predicate-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/predicate-functions.sql.out
index 8e7e04c8e1c4f..5d91be3e72913 100644
--- a/sql/core/src/test/resources/sql-tests/results/predicate-functions.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/predicate-functions.sql.out
@@ -13,7 +13,7 @@ true
-- !query 1
select 1 = '1'
-- !query 1 schema
-struct<(1 = CAST(1 AS INT)):boolean>
+struct<(CAST(1 AS DOUBLE) = CAST(1 AS DOUBLE)):boolean>
-- !query 1 output
true
@@ -21,7 +21,7 @@ true
-- !query 2
select 1.0 = '1'
-- !query 2 schema
-struct<(1.0 = CAST(1 AS DECIMAL(2,1))):boolean>
+struct<(CAST(1.0 AS DOUBLE) = CAST(1 AS DOUBLE)):boolean>
-- !query 2 output
true
@@ -29,7 +29,7 @@ true
-- !query 3
select 1 > '1'
-- !query 3 schema
-struct<(1 > CAST(1 AS INT)):boolean>
+struct<(CAST(1 AS DOUBLE) > CAST(1 AS DOUBLE)):boolean>
-- !query 3 output
false
@@ -37,7 +37,7 @@ false
-- !query 4
select 2 > '1.0'
-- !query 4 schema
-struct<(2 > CAST(1.0 AS INT)):boolean>
+struct<(CAST(2 AS DOUBLE) > CAST(1.0 AS DOUBLE)):boolean>
-- !query 4 output
true
@@ -45,7 +45,7 @@ true
-- !query 5
select 2 > '2.0'
-- !query 5 schema
-struct<(2 > CAST(2.0 AS INT)):boolean>
+struct<(CAST(2 AS DOUBLE) > CAST(2.0 AS DOUBLE)):boolean>
-- !query 5 output
false
@@ -53,7 +53,7 @@ false
-- !query 6
select 2 > '2.2'
-- !query 6 schema
-struct<(2 > CAST(2.2 AS INT)):boolean>
+struct<(CAST(2 AS DOUBLE) > CAST(2.2 AS DOUBLE)):boolean>
-- !query 6 output
false
@@ -69,7 +69,7 @@ false
-- !query 8
select to_date('2009-07-30 04:17:52') > '2009-07-30 04:17:52'
-- !query 8 schema
-struct<(CAST(to_date('2009-07-30 04:17:52') AS STRING) > 2009-07-30 04:17:52):boolean>
+struct<(to_date('2009-07-30 04:17:52') > CAST(2009-07-30 04:17:52 AS DATE)):boolean>
-- !query 8 output
false
@@ -77,7 +77,7 @@ false
-- !query 9
select 1 >= '1'
-- !query 9 schema
-struct<(1 >= CAST(1 AS INT)):boolean>
+struct<(CAST(1 AS DOUBLE) >= CAST(1 AS DOUBLE)):boolean>
-- !query 9 output
true
@@ -85,7 +85,7 @@ true
-- !query 10
select 2 >= '1.0'
-- !query 10 schema
-struct<(2 >= CAST(1.0 AS INT)):boolean>
+struct<(CAST(2 AS DOUBLE) >= CAST(1.0 AS DOUBLE)):boolean>
-- !query 10 output
true
@@ -93,7 +93,7 @@ true
-- !query 11
select 2 >= '2.0'
-- !query 11 schema
-struct<(2 >= CAST(2.0 AS INT)):boolean>
+struct<(CAST(2 AS DOUBLE) >= CAST(2.0 AS DOUBLE)):boolean>
-- !query 11 output
true
@@ -101,7 +101,7 @@ true
-- !query 12
select 2.0 >= '2.2'
-- !query 12 schema
-struct<(2.0 >= CAST(2.2 AS DECIMAL(2,1))):boolean>
+struct<(CAST(2.0 AS DOUBLE) >= CAST(2.2 AS DOUBLE)):boolean>
-- !query 12 output
false
@@ -117,15 +117,15 @@ true
-- !query 14
select to_date('2009-07-30 04:17:52') >= '2009-07-30 04:17:52'
-- !query 14 schema
-struct<(CAST(to_date('2009-07-30 04:17:52') AS STRING) >= 2009-07-30 04:17:52):boolean>
+struct<(to_date('2009-07-30 04:17:52') >= CAST(2009-07-30 04:17:52 AS DATE)):boolean>
-- !query 14 output
-false
+true
-- !query 15
select 1 < '1'
-- !query 15 schema
-struct<(1 < CAST(1 AS INT)):boolean>
+struct<(CAST(1 AS DOUBLE) < CAST(1 AS DOUBLE)):boolean>
-- !query 15 output
false
@@ -133,7 +133,7 @@ false
-- !query 16
select 2 < '1.0'
-- !query 16 schema
-struct<(2 < CAST(1.0 AS INT)):boolean>
+struct<(CAST(2 AS DOUBLE) < CAST(1.0 AS DOUBLE)):boolean>
-- !query 16 output
false
@@ -141,7 +141,7 @@ false
-- !query 17
select 2 < '2.0'
-- !query 17 schema
-struct<(2 < CAST(2.0 AS INT)):boolean>
+struct<(CAST(2 AS DOUBLE) < CAST(2.0 AS DOUBLE)):boolean>
-- !query 17 output
false
@@ -149,7 +149,7 @@ false
-- !query 18
select 2.0 < '2.2'
-- !query 18 schema
-struct<(2.0 < CAST(2.2 AS DECIMAL(2,1))):boolean>
+struct<(CAST(2.0 AS DOUBLE) < CAST(2.2 AS DOUBLE)):boolean>
-- !query 18 output
true
@@ -165,15 +165,15 @@ false
-- !query 20
select to_date('2009-07-30 04:17:52') < '2009-07-30 04:17:52'
-- !query 20 schema
-struct<(CAST(to_date('2009-07-30 04:17:52') AS STRING) < 2009-07-30 04:17:52):boolean>
+struct<(to_date('2009-07-30 04:17:52') < CAST(2009-07-30 04:17:52 AS DATE)):boolean>
-- !query 20 output
-true
+false
-- !query 21
select 1 <= '1'
-- !query 21 schema
-struct<(1 <= CAST(1 AS INT)):boolean>
+struct<(CAST(1 AS DOUBLE) <= CAST(1 AS DOUBLE)):boolean>
-- !query 21 output
true
@@ -181,7 +181,7 @@ true
-- !query 22
select 2 <= '1.0'
-- !query 22 schema
-struct<(2 <= CAST(1.0 AS INT)):boolean>
+struct<(CAST(2 AS DOUBLE) <= CAST(1.0 AS DOUBLE)):boolean>
-- !query 22 output
false
@@ -189,7 +189,7 @@ false
-- !query 23
select 2 <= '2.0'
-- !query 23 schema
-struct<(2 <= CAST(2.0 AS INT)):boolean>
+struct<(CAST(2 AS DOUBLE) <= CAST(2.0 AS DOUBLE)):boolean>
-- !query 23 output
true
@@ -197,7 +197,7 @@ true
-- !query 24
select 2.0 <= '2.2'
-- !query 24 schema
-struct<(2.0 <= CAST(2.2 AS DECIMAL(2,1))):boolean>
+struct<(CAST(2.0 AS DOUBLE) <= CAST(2.2 AS DOUBLE)):boolean>
-- !query 24 output
true
@@ -213,6 +213,6 @@ true
-- !query 26
select to_date('2009-07-30 04:17:52') <= '2009-07-30 04:17:52'
-- !query 26 schema
-struct<(CAST(to_date('2009-07-30 04:17:52') AS STRING) <= 2009-07-30 04:17:52):boolean>
+struct<(to_date('2009-07-30 04:17:52') <= CAST(2009-07-30 04:17:52 AS DATE)):boolean>
-- !query 26 output
true
From 844aec7f4022140921d485b89fc240846bd05ac3 Mon Sep 17 00:00:00 2001
From: Yuming Wang
Date: Mon, 18 Sep 2017 15:27:13 +0800
Subject: [PATCH 06/19] Add spark.sql.binary.comparison.compatible.with.hive
conf.
---
docs/sql-programming-guide.md | 7 +++
.../sql/catalyst/analysis/TypeCoercion.scala | 43 ++++++++++++++++---
.../apache/spark/sql/internal/SQLConf.scala | 9 ++++
.../org/apache/spark/sql/SQLQuerySuite.scala | 34 ++++++++++++---
4 files changed, 80 insertions(+), 13 deletions(-)
diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md
index 45ba4d168f043..b9d5cffadf6f3 100644
--- a/docs/sql-programming-guide.md
+++ b/docs/sql-programming-guide.md
@@ -968,6 +968,13 @@ Configuration of Parquet can be done using the `setConf` method on `SparkSession
+
+ spark.sql.binary.comparison.compatible.with.hive |
+ true |
+
+ Whether compatible with Hive when binary comparison.
+ |
+
## JSON Datasets
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 90e8d8e54ccd8..296908b60a149 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
@@ -116,9 +116,29 @@ object TypeCoercion {
* other is a Timestamp by making the target type to be String.
*/
val findCommonTypeForBinaryComparison: (DataType, DataType) => Option[DataType] = {
- // We should follow hive:
- // https://github.com/apache/hive/blob/rel/storage-release-2.4.0/ql/src/java/
- // org/apache/hadoop/hive/ql/exec/FunctionRegistry.java#L781
+ // We should cast all relative timestamp/date/string comparison into string comparisons
+ // This behaves as a user would expect because timestamp strings sort lexicographically.
+ // i.e. TimeStamp(2013-01-01 00:00 ...) < "2014" = true
+ case (StringType, DateType) => Some(StringType)
+ case (DateType, StringType) => Some(StringType)
+ case (StringType, TimestampType) => Some(StringType)
+ case (TimestampType, StringType) => Some(StringType)
+ case (TimestampType, DateType) => Some(StringType)
+ case (DateType, TimestampType) => Some(StringType)
+ case (StringType, NullType) => Some(StringType)
+ case (NullType, StringType) => Some(StringType)
+ case (l: StringType, r: AtomicType) if r != StringType => Some(r)
+ case (l: AtomicType, r: StringType) if (l != StringType) => Some(l)
+ case (l, r) => None
+ }
+
+ /**
+ * Follow hive's binary comparison action:
+ * https://github.com/apache/hive/blob/rel/storage-release-2.4.0/ql/src/java/
+ * org/apache/hadoop/hive/ql/exec/FunctionRegistry.java#L781
+ */
+ val findCommonTypeCompatibleWithHive: (DataType, DataType) =>
+ Option[DataType] = {
case (StringType, DateType) => Some(DateType)
case (DateType, StringType) => Some(DateType)
case (StringType, TimestampType) => Some(TimestampType)
@@ -355,9 +375,15 @@ object TypeCoercion {
case p @ Equality(left @ TimestampType(), right @ StringType()) =>
p.makeCopy(Array(left, Cast(right, TimestampType)))
case p @ BinaryComparison(left, right)
- if findCommonTypeForBinaryComparison(left.dataType, right.dataType).isDefined =>
+ if !plan.conf.binaryComparisonCompatibleWithHive &&
+ findCommonTypeForBinaryComparison(left.dataType, right.dataType).isDefined =>
val commonType = findCommonTypeForBinaryComparison(left.dataType, right.dataType).get
p.makeCopy(Array(castExpr(left, commonType), castExpr(right, commonType)))
+ case p @ BinaryComparison(left, right)
+ if plan.conf.binaryComparisonCompatibleWithHive &&
+ findCommonTypeCompatibleWithHive(left.dataType, right.dataType).isDefined =>
+ val commonType = findCommonTypeCompatibleWithHive(left.dataType, right.dataType).get
+ p.makeCopy(Array(castExpr(left, commonType), castExpr(right, commonType)))
case Abs(e @ StringType()) => Abs(Cast(e, DoubleType))
case Sum(e @ StringType()) => Sum(Cast(e, DoubleType))
@@ -412,8 +438,13 @@ object TypeCoercion {
val rhs = sub.output
val commonTypes = lhs.zip(rhs).flatMap { case (l, r) =>
- findCommonTypeForBinaryComparison(l.dataType, r.dataType)
- .orElse(findTightestCommonType(l.dataType, r.dataType))
+ if (plan.conf.binaryComparisonCompatibleWithHive) {
+ findCommonTypeCompatibleWithHive(l.dataType, r.dataType)
+ .orElse(findTightestCommonType(l.dataType, r.dataType))
+ } else {
+ findCommonTypeForBinaryComparison(l.dataType, r.dataType)
+ .orElse(findTightestCommonType(l.dataType, r.dataType))
+ }
}
// The number of columns/expressions must match between LHS and RHS of an
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
index d00c672487532..8b0380b980e3b 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
@@ -925,6 +925,12 @@ object SQLConf {
.intConf
.createWithDefault(10000)
+ val BINARY_COMPARISON_COMPATIBLE_WITH_HIVE =
+ buildConf("spark.sql.binary.comparison.compatible.with.hive")
+ .doc("Whether compatible with Hive when binary comparison.")
+ .booleanConf
+ .createWithDefault(true)
+
object Deprecated {
val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks"
}
@@ -1203,6 +1209,9 @@ class SQLConf extends Serializable with Logging {
def arrowMaxRecordsPerBatch: Int = getConf(ARROW_EXECUTION_MAX_RECORDS_PER_BATCH)
+ def binaryComparisonCompatibleWithHive: Boolean =
+ getConf(SQLConf.BINARY_COMPARISON_COMPATIBLE_WITH_HIVE)
+
/** ********************** SQLConf functionality methods ************ */
/** Set Spark SQL configuration properties. */
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
index bfd9acb068634..5146623862898 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
@@ -2684,18 +2684,38 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
val str2 = Int.MaxValue.toString + "1"
val str3 = "10"
Seq(str1, str2, str3).toDF("c1").createOrReplaceTempView("v")
- checkAnswer(sql("SELECT c1 from v where c1 > 0"), Row(str1) :: Row(str2) :: Row(str3) :: Nil)
- checkAnswer(sql("SELECT c1 from v where c1 > 0L"), Row(str1) :: Row(str2) :: Row(str3) :: Nil)
+ withSQLConf(SQLConf.BINARY_COMPARISON_COMPATIBLE_WITH_HIVE.key -> "true") {
+ checkAnswer(sql("SELECT c1 from v where c1 > 0"),
+ Row(str1) :: Row(str2) :: Row(str3) :: Nil)
+ checkAnswer(sql("SELECT c1 from v where c1 > 0L"),
+ Row(str1) :: Row(str2) :: Row(str3) :: Nil)
+ }
+
+ withSQLConf(SQLConf.BINARY_COMPARISON_COMPATIBLE_WITH_HIVE.key -> "false") {
+ checkAnswer(sql("SELECT c1 from v where c1 > 0"), Row(str3) :: Nil)
+ checkAnswer(sql("SELECT c1 from v where c1 > 0L"), Row(str2) :: Row(str3) :: Nil)
+ }
}
}
test("SPARK-21646: CommonTypeForBinaryComparison: DoubleType vs IntegerType") {
withTempView("v") {
- Seq(("0", 1), ("-0.4", 2)).toDF("a", "b").createOrReplaceTempView("v")
- checkAnswer(sql("SELECT a FROM v WHERE a=0"), Seq(Row("0")))
- checkAnswer(sql("SELECT a FROM v WHERE a=0L"), Seq(Row("0")))
- checkAnswer(sql("SELECT a FROM v WHERE a=0.0"), Seq(Row("0")))
- checkAnswer(sql("SELECT a FROM v WHERE a=-0.4"), Seq(Row("-0.4")))
+ Seq(("0", 1), ("-0.4", 2), ("0.6", 3)).toDF("a", "b").createOrReplaceTempView("v")
+ withSQLConf(SQLConf.BINARY_COMPARISON_COMPATIBLE_WITH_HIVE.key -> "true") {
+ checkAnswer(sql("SELECT a FROM v WHERE a = 0"), Seq(Row("0")))
+ checkAnswer(sql("SELECT a FROM v WHERE a = 0L"), Seq(Row("0")))
+ checkAnswer(sql("SELECT a FROM v WHERE a = 0.0"), Seq(Row("0")))
+ checkAnswer(sql("SELECT a FROM v WHERE a = -0.4"), Seq(Row("-0.4")))
+ checkAnswer(sql("SELECT count(*) FROM v WHERE a > 0"), Row(1) :: Nil)
+ }
+
+ withSQLConf(SQLConf.BINARY_COMPARISON_COMPATIBLE_WITH_HIVE.key -> "false") {
+ checkAnswer(sql("SELECT a FROM v WHERE a = 0"), Seq(Row("0"), Row("-0.4"), Row("0.6")))
+ checkAnswer(sql("SELECT a FROM v WHERE a = 0L"), Seq(Row("0"), Row("-0.4"), Row("0.6")))
+ checkAnswer(sql("SELECT a FROM v WHERE a = 0.0"), Seq(Row("0")))
+ checkAnswer(sql("SELECT a FROM v WHERE a = -0.4"), Seq(Row("-0.4")))
+ checkAnswer(sql("SELECT count(*) FROM v WHERE a > 0"), Row(0) :: Nil)
+ }
}
}
From 7812018b41957dbc38bf30f3e31c9c2c49b22c1c Mon Sep 17 00:00:00 2001
From: Yuming Wang
Date: Tue, 19 Sep 2017 23:00:24 +0800
Subject: [PATCH 07/19] spark.sql.binary.comparison.compatible.with.hive ->
spark.sql.autoTypeCastingCompatibility
---
docs/sql-programming-guide.md | 6 +-
.../sql/catalyst/analysis/TypeCoercion.scala | 95 ++++++++--------
.../apache/spark/sql/internal/SQLConf.scala | 12 +--
.../org/apache/spark/sql/SQLQuerySuite.scala | 102 +++++++++++++-----
4 files changed, 127 insertions(+), 88 deletions(-)
diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md
index b9d5cffadf6f3..1f4c875282ba2 100644
--- a/docs/sql-programming-guide.md
+++ b/docs/sql-programming-guide.md
@@ -969,10 +969,10 @@ Configuration of Parquet can be done using the `setConf` method on `SparkSession
- spark.sql.binary.comparison.compatible.with.hive |
- true |
+ spark.sql.autoTypeCastingCompatibility |
+ false |
- Whether compatible with Hive when binary comparison.
+ Whether compatible with Hive.
|
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 296908b60a149..b013c311469a5 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
@@ -115,43 +115,46 @@ object TypeCoercion {
* is a String and the other is not. It also handles when one op is a Date and the
* other is a Timestamp by making the target type to be String.
*/
- val findCommonTypeForBinaryComparison: (DataType, DataType) => Option[DataType] = {
- // We should cast all relative timestamp/date/string comparison into string comparisons
- // This behaves as a user would expect because timestamp strings sort lexicographically.
- // i.e. TimeStamp(2013-01-01 00:00 ...) < "2014" = true
- case (StringType, DateType) => Some(StringType)
- case (DateType, StringType) => Some(StringType)
- case (StringType, TimestampType) => Some(StringType)
- case (TimestampType, StringType) => Some(StringType)
- case (TimestampType, DateType) => Some(StringType)
- case (DateType, TimestampType) => Some(StringType)
- case (StringType, NullType) => Some(StringType)
- case (NullType, StringType) => Some(StringType)
- case (l: StringType, r: AtomicType) if r != StringType => Some(r)
- case (l: AtomicType, r: StringType) if (l != StringType) => Some(l)
- case (l, r) => None
- }
-
- /**
- * Follow hive's binary comparison action:
- * https://github.com/apache/hive/blob/rel/storage-release-2.4.0/ql/src/java/
- * org/apache/hadoop/hive/ql/exec/FunctionRegistry.java#L781
- */
- val findCommonTypeCompatibleWithHive: (DataType, DataType) =>
- Option[DataType] = {
- case (StringType, DateType) => Some(DateType)
- case (DateType, StringType) => Some(DateType)
- case (StringType, TimestampType) => Some(TimestampType)
- case (TimestampType, StringType) => Some(TimestampType)
- case (TimestampType, DateType) => Some(TimestampType)
- case (DateType, TimestampType) => Some(TimestampType)
- case (StringType, NullType) => Some(StringType)
- case (NullType, StringType) => Some(StringType)
- case (StringType | TimestampType, r: NumericType) => Some(DoubleType)
- case (l: NumericType, StringType | TimestampType) => Some(DoubleType)
- case (l: StringType, r: AtomicType) if r != StringType => Some(r)
- case (l: AtomicType, r: StringType) if l != StringType => Some(l)
- case (l, r) => None
+ private def findCommonTypeForBinaryComparison(
+ plan: LogicalPlan,
+ l: DataType,
+ r: DataType): Option[DataType] =
+ if (!plan.conf.autoTypeCastingCompatibility) {
+ (l, r) match {
+ // We should cast all relative timestamp/date/string comparison into string comparisons
+ // This behaves as a user would expect because timestamp strings sort lexicographically.
+ // i.e. TimeStamp(2013-01-01 00:00 ...) < "2014" = true
+ case (StringType, DateType) => Some(StringType)
+ case (DateType, StringType) => Some(StringType)
+ case (StringType, TimestampType) => Some(StringType)
+ case (TimestampType, StringType) => Some(StringType)
+ case (TimestampType, DateType) => Some(StringType)
+ case (DateType, TimestampType) => Some(StringType)
+ case (StringType, NullType) => Some(StringType)
+ case (NullType, StringType) => Some(StringType)
+ case (l: StringType, r: AtomicType) if r != StringType => Some(r)
+ case (l: AtomicType, r: StringType) if (l != StringType) => Some(l)
+ case (l, r) => None
+ }
+ } else {
+ (l, r) match {
+ // Follow hive's binary comparison action:
+ // https://github.com/apache/hive/blob/rel/storage-release-2.4.0/ql/src/java/
+ // org/apache/hadoop/hive/ql/exec/FunctionRegistry.java#L781
+ case (StringType, DateType) => Some(DateType)
+ case (DateType, StringType) => Some(DateType)
+ case (StringType, TimestampType) => Some(TimestampType)
+ case (TimestampType, StringType) => Some(TimestampType)
+ case (TimestampType, DateType) => Some(TimestampType)
+ case (DateType, TimestampType) => Some(TimestampType)
+ case (StringType, NullType) => Some(StringType)
+ case (NullType, StringType) => Some(StringType)
+ case (StringType | TimestampType, r: NumericType) => Some(DoubleType)
+ case (l: NumericType, StringType | TimestampType) => Some(DoubleType)
+ case (l: StringType, r: AtomicType) if r != StringType => Some(r)
+ case (l: AtomicType, r: StringType) if l != StringType => Some(l)
+ case _ => None
+ }
}
/**
@@ -375,14 +378,8 @@ object TypeCoercion {
case p @ Equality(left @ TimestampType(), right @ StringType()) =>
p.makeCopy(Array(left, Cast(right, TimestampType)))
case p @ BinaryComparison(left, right)
- if !plan.conf.binaryComparisonCompatibleWithHive &&
- findCommonTypeForBinaryComparison(left.dataType, right.dataType).isDefined =>
- val commonType = findCommonTypeForBinaryComparison(left.dataType, right.dataType).get
- p.makeCopy(Array(castExpr(left, commonType), castExpr(right, commonType)))
- case p @ BinaryComparison(left, right)
- if plan.conf.binaryComparisonCompatibleWithHive &&
- findCommonTypeCompatibleWithHive(left.dataType, right.dataType).isDefined =>
- val commonType = findCommonTypeCompatibleWithHive(left.dataType, right.dataType).get
+ if findCommonTypeForBinaryComparison(plan, left.dataType, right.dataType).isDefined =>
+ val commonType = findCommonTypeForBinaryComparison(plan, left.dataType, right.dataType).get
p.makeCopy(Array(castExpr(left, commonType), castExpr(right, commonType)))
case Abs(e @ StringType()) => Abs(Cast(e, DoubleType))
@@ -438,13 +435,9 @@ object TypeCoercion {
val rhs = sub.output
val commonTypes = lhs.zip(rhs).flatMap { case (l, r) =>
- if (plan.conf.binaryComparisonCompatibleWithHive) {
- findCommonTypeCompatibleWithHive(l.dataType, r.dataType)
- .orElse(findTightestCommonType(l.dataType, r.dataType))
- } else {
- findCommonTypeForBinaryComparison(l.dataType, r.dataType)
+ findCommonTypeForBinaryComparison(plan, l.dataType, r.dataType)
.orElse(findTightestCommonType(l.dataType, r.dataType))
- }
+
}
// The number of columns/expressions must match between LHS and RHS of an
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
index 8b0380b980e3b..d2a5ea9f02870 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
@@ -925,11 +925,11 @@ object SQLConf {
.intConf
.createWithDefault(10000)
- val BINARY_COMPARISON_COMPATIBLE_WITH_HIVE =
- buildConf("spark.sql.binary.comparison.compatible.with.hive")
- .doc("Whether compatible with Hive when binary comparison.")
+ val AUTO_TYPE_CASTING_COMPATIBILITY =
+ buildConf("spark.sql.autoTypeCastingCompatibility")
+ .doc("Whether compatible with Hive.")
.booleanConf
- .createWithDefault(true)
+ .createWithDefault(false)
object Deprecated {
val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks"
@@ -1209,8 +1209,8 @@ class SQLConf extends Serializable with Logging {
def arrowMaxRecordsPerBatch: Int = getConf(ARROW_EXECUTION_MAX_RECORDS_PER_BATCH)
- def binaryComparisonCompatibleWithHive: Boolean =
- getConf(SQLConf.BINARY_COMPARISON_COMPATIBLE_WITH_HIVE)
+ def autoTypeCastingCompatibility: Boolean =
+ getConf(SQLConf.AUTO_TYPE_CASTING_COMPATIBILITY)
/** ********************** SQLConf functionality methods ************ */
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
index 5146623862898..137ca113be667 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
@@ -2684,14 +2684,14 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
val str2 = Int.MaxValue.toString + "1"
val str3 = "10"
Seq(str1, str2, str3).toDF("c1").createOrReplaceTempView("v")
- withSQLConf(SQLConf.BINARY_COMPARISON_COMPATIBLE_WITH_HIVE.key -> "true") {
+ withSQLConf(SQLConf.AUTO_TYPE_CASTING_COMPATIBILITY.key -> "true") {
checkAnswer(sql("SELECT c1 from v where c1 > 0"),
Row(str1) :: Row(str2) :: Row(str3) :: Nil)
checkAnswer(sql("SELECT c1 from v where c1 > 0L"),
Row(str1) :: Row(str2) :: Row(str3) :: Nil)
}
- withSQLConf(SQLConf.BINARY_COMPARISON_COMPATIBLE_WITH_HIVE.key -> "false") {
+ withSQLConf(SQLConf.AUTO_TYPE_CASTING_COMPATIBILITY.key -> "false") {
checkAnswer(sql("SELECT c1 from v where c1 > 0"), Row(str3) :: Nil)
checkAnswer(sql("SELECT c1 from v where c1 > 0L"), Row(str2) :: Row(str3) :: Nil)
}
@@ -2700,21 +2700,21 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
test("SPARK-21646: CommonTypeForBinaryComparison: DoubleType vs IntegerType") {
withTempView("v") {
- Seq(("0", 1), ("-0.4", 2), ("0.6", 3)).toDF("a", "b").createOrReplaceTempView("v")
- withSQLConf(SQLConf.BINARY_COMPARISON_COMPATIBLE_WITH_HIVE.key -> "true") {
- checkAnswer(sql("SELECT a FROM v WHERE a = 0"), Seq(Row("0")))
- checkAnswer(sql("SELECT a FROM v WHERE a = 0L"), Seq(Row("0")))
- checkAnswer(sql("SELECT a FROM v WHERE a = 0.0"), Seq(Row("0")))
- checkAnswer(sql("SELECT a FROM v WHERE a = -0.4"), Seq(Row("-0.4")))
- checkAnswer(sql("SELECT count(*) FROM v WHERE a > 0"), Row(1) :: Nil)
+ Seq(("0", 1), ("-0.4", 2), ("0.6", 3)).toDF("c1", "c2").createOrReplaceTempView("v")
+ withSQLConf(SQLConf.AUTO_TYPE_CASTING_COMPATIBILITY.key -> "true") {
+ checkAnswer(sql("SELECT c1 FROM v WHERE c1 = 0"), Seq(Row("0")))
+ checkAnswer(sql("SELECT c1 FROM v WHERE c1 = 0L"), Seq(Row("0")))
+ checkAnswer(sql("SELECT c1 FROM v WHERE c1 = 0.0"), Seq(Row("0")))
+ checkAnswer(sql("SELECT c1 FROM v WHERE c1 = -0.4"), Seq(Row("-0.4")))
+ checkAnswer(sql("SELECT count(*) FROM v WHERE c1 > 0"), Row(1) :: Nil)
}
- withSQLConf(SQLConf.BINARY_COMPARISON_COMPATIBLE_WITH_HIVE.key -> "false") {
- checkAnswer(sql("SELECT a FROM v WHERE a = 0"), Seq(Row("0"), Row("-0.4"), Row("0.6")))
- checkAnswer(sql("SELECT a FROM v WHERE a = 0L"), Seq(Row("0"), Row("-0.4"), Row("0.6")))
- checkAnswer(sql("SELECT a FROM v WHERE a = 0.0"), Seq(Row("0")))
- checkAnswer(sql("SELECT a FROM v WHERE a = -0.4"), Seq(Row("-0.4")))
- checkAnswer(sql("SELECT count(*) FROM v WHERE a > 0"), Row(0) :: Nil)
+ withSQLConf(SQLConf.AUTO_TYPE_CASTING_COMPATIBILITY.key -> "false") {
+ checkAnswer(sql("SELECT c1 FROM v WHERE c1 = 0"), Seq(Row("0"), Row("-0.4"), Row("0.6")))
+ checkAnswer(sql("SELECT c1 FROM v WHERE c1 = 0L"), Seq(Row("0"), Row("-0.4"), Row("0.6")))
+ checkAnswer(sql("SELECT c1 FROM v WHERE c1 = 0.0"), Seq(Row("0")))
+ checkAnswer(sql("SELECT c1 FROM v WHERE c1 = -0.4"), Seq(Row("-0.4")))
+ checkAnswer(sql("SELECT count(*) FROM v WHERE c1 > 0"), Row(0) :: Nil)
}
}
}
@@ -2724,9 +2724,17 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
val v1 = Date.valueOf("2017-09-22")
val v2 = Date.valueOf("2017-09-09")
Seq(v1, v2).toDF("c1").createTempView("v")
- checkAnswer(sql("select * from v where c1 > '2017-8-1'"), Row(v1) :: Row(v2) :: Nil)
- checkAnswer(sql("select * from v where c1 > cast('2017-8-1' as date)"),
- Row(v1) :: Row(v2) :: Nil)
+ withSQLConf(SQLConf.AUTO_TYPE_CASTING_COMPATIBILITY.key -> "true") {
+ checkAnswer(sql("select c1 from v where c1 > '2017-8-1'"), Row(v1) :: Row(v2) :: Nil)
+ checkAnswer(sql("select c1 from v where c1 > cast('2017-8-1' as date)"),
+ Row(v1) :: Row(v2) :: Nil)
+ }
+
+ withSQLConf(SQLConf.AUTO_TYPE_CASTING_COMPATIBILITY.key -> "false") {
+ checkAnswer(sql("select c1 from v where c1 > '2017-8-1'"), Nil)
+ checkAnswer(sql("select c1 from v where c1 > cast('2017-8-1' as date)"),
+ Row(v1) :: Row(v2) :: Nil)
+ }
}
}
@@ -2734,9 +2742,18 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
withTempView("v") {
val v1 = Timestamp.valueOf("2017-07-21 23:42:12.123")
val v2 = Timestamp.valueOf("2017-08-21 23:42:12.123")
- val df = Seq(v1, v2).toDF("c1").createTempView("v")
- checkAnswer(sql("select * from v where c1 > '2017-8-1'"), Row(v2) :: Nil)
- checkAnswer(sql("select * from v where c1 > cast('2017-8-1' as timestamp)"), Row(v2) :: Nil)
+ Seq(v1, v2).toDF("c1").createTempView("v")
+ withSQLConf(SQLConf.AUTO_TYPE_CASTING_COMPATIBILITY.key -> "true") {
+ checkAnswer(sql("select c1 from v where c1 > '2017-8-1'"), Row(v2) :: Nil)
+ checkAnswer(sql("select c1 from v where c1 > cast('2017-8-1' as timestamp)"),
+ Row(v2) :: Nil)
+ }
+
+ withSQLConf(SQLConf.AUTO_TYPE_CASTING_COMPATIBILITY.key -> "false") {
+ checkAnswer(sql("select c1 from v where c1 > '2017-8-1'"), Nil)
+ checkAnswer(sql("select c1 from v where c1 > cast('2017-8-1' as timestamp)"),
+ Row(v2) :: Nil)
+ }
}
}
@@ -2744,9 +2761,18 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
withTempView("v") {
val v1 = Timestamp.valueOf("2017-07-21 23:42:12.123")
val v2 = Timestamp.valueOf("2017-08-21 23:42:12.123")
- val df = Seq(v1, v2).toDF("c1").createTempView("v")
- checkAnswer(sql("select * from v where c1 > cast('2017-8-1' as date)"), Row(v2) :: Nil)
- checkAnswer(sql("select * from v where c1 > cast('2017-8-1' as timestamp)"), Row(v2) :: Nil)
+ Seq(v1, v2).toDF("c1").createTempView("v")
+ withSQLConf(SQLConf.AUTO_TYPE_CASTING_COMPATIBILITY.key -> "true") {
+ checkAnswer(sql("select c1 from v where c1 > cast('2017-8-1' as date)"), Row(v2) :: Nil)
+ checkAnswer(sql("select c1 from v where c1 > cast('2017-8-1' as timestamp)"),
+ Row(v2) :: Nil)
+ }
+
+ withSQLConf(SQLConf.AUTO_TYPE_CASTING_COMPATIBILITY.key -> "false") {
+ checkAnswer(sql("select c1 from v where c1 > cast('2017-8-1' as date)"), Row(v2) :: Nil)
+ checkAnswer(sql("select c1 from v where c1 > cast('2017-8-1' as timestamp)"),
+ Row(v2) :: Nil)
+ }
}
}
@@ -2754,10 +2780,30 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
withTempView("v") {
val v1 = Timestamp.valueOf("2017-07-21 23:42:12.123")
val v2 = Timestamp.valueOf("2017-08-21 23:42:12.123")
- val df = Seq(v1, v2).toDF("c1").createTempView("v")
- checkAnswer(sql("select * from v where c1 > 1"), Row(v1) :: Row(v2) :: Nil)
- checkAnswer(sql("select * from v where c1 > cast(cast('2010-08-01' as timestamp) as double)"),
- Row(v1) :: Row(v2) :: Nil)
+ Seq(v1, v2).toDF("c1").createTempView("v")
+ withSQLConf(SQLConf.AUTO_TYPE_CASTING_COMPATIBILITY.key -> "true") {
+ checkAnswer(sql("select c1 from v where c1 > 1"), Row(v1) :: Row(v2) :: Nil)
+ checkAnswer(sql("select c1 from v where c1 > '2017-8-1'"), Row(v2) :: Nil)
+ checkAnswer(sql("select c1 from v where c1 > '2017-08-01'"), Row(v2) :: Nil)
+ checkAnswer(
+ sql("select * from v where c1 > cast(cast('2017-08-01' as timestamp) as double)"),
+ Row(v2) :: Nil)
+ }
+
+ withSQLConf(SQLConf.AUTO_TYPE_CASTING_COMPATIBILITY.key -> "false") {
+ val e1 = intercept[AnalysisException] {
+ sql("select * from v where c1 > 1")
+ }
+ assert(e1.getMessage.contains("data type mismatch"))
+
+ checkAnswer(sql("select c1 from v where c1 > '2017-8-1'"), Nil)
+ checkAnswer(sql("select c1 from v where c1 > '2017-08-01'"), Row(v2) :: Nil)
+
+ val e2 = intercept[AnalysisException] {
+ sql("select * from v where c1 > cast(cast('2017-08-01' as timestamp) as double)")
+ }
+ assert(e2.getMessage.contains("data type mismatch"))
+ }
}
}
}
From 27d5b13190267637ed9dae8d7cb6aa762d7fa320 Mon Sep 17 00:00:00 2001
From: Yuming Wang
Date: Wed, 20 Sep 2017 20:32:51 +0800
Subject: [PATCH 08/19] spark.sql.autoTypeCastingCompatibility ->
spark.sql.typeCoercion.mode
---
docs/sql-programming-guide.md | 14 +++---
.../sql/catalyst/analysis/TypeCoercion.scala | 2 +-
.../apache/spark/sql/internal/SQLConf.scala | 13 ++---
.../catalyst/analysis/TypeCoercionSuite.scala | 25 +---------
.../results/predicate-functions.sql.out | 48 +++++++++----------
.../org/apache/spark/sql/DataFrameSuite.scala | 2 +-
.../org/apache/spark/sql/SQLQuerySuite.scala | 37 ++++++++------
7 files changed, 65 insertions(+), 76 deletions(-)
diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md
index 1f4c875282ba2..6ada6cf5cdde5 100644
--- a/docs/sql-programming-guide.md
+++ b/docs/sql-programming-guide.md
@@ -968,13 +968,6 @@ Configuration of Parquet can be done using the `setConf` method on `SparkSession
-
- spark.sql.autoTypeCastingCompatibility |
- false |
-
- Whether compatible with Hive.
- |
-
## JSON Datasets
@@ -1467,6 +1460,13 @@ that these options will be deprecated in future release as more optimizations ar
Configures the number of partitions to use when shuffling data for joins or aggregations.
+
+ spark.sql.typeCoercion.mode |
+ default |
+
+ Whether compatible with Hive. Available options are default and hive.
+ |
+
# Distributed SQL Engine
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 b013c311469a5..d6094cf5a5bf0 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
@@ -119,7 +119,7 @@ object TypeCoercion {
plan: LogicalPlan,
l: DataType,
r: DataType): Option[DataType] =
- if (!plan.conf.autoTypeCastingCompatibility) {
+ if (!plan.conf.isHiveTypeCoercionMode) {
(l, r) match {
// We should cast all relative timestamp/date/string comparison into string comparisons
// This behaves as a user would expect because timestamp strings sort lexicographically.
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
index d2a5ea9f02870..ee22e1823b516 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
@@ -925,11 +925,13 @@ object SQLConf {
.intConf
.createWithDefault(10000)
- val AUTO_TYPE_CASTING_COMPATIBILITY =
- buildConf("spark.sql.autoTypeCastingCompatibility")
+ val typeCoercionMode =
+ buildConf("spark.sql.typeCoercion.mode")
.doc("Whether compatible with Hive.")
- .booleanConf
- .createWithDefault(false)
+ .stringConf
+ .transform(_.toLowerCase(Locale.ROOT))
+ .checkValues(Set("default", "hive"))
+ .createWithDefault("default")
object Deprecated {
val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks"
@@ -1209,8 +1211,7 @@ class SQLConf extends Serializable with Logging {
def arrowMaxRecordsPerBatch: Int = getConf(ARROW_EXECUTION_MAX_RECORDS_PER_BATCH)
- def autoTypeCastingCompatibility: Boolean =
- getConf(SQLConf.AUTO_TYPE_CASTING_COMPATIBILITY)
+ def isHiveTypeCoercionMode: Boolean = getConf(SQLConf.typeCoercionMode).equals("hive")
/** ********************** SQLConf functionality methods ************ */
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 d93a3d957fbe5..d62e3b6dfe34f 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
@@ -17,7 +17,7 @@
package org.apache.spark.sql.catalyst.analysis
-import java.sql.{Date, Timestamp}
+import java.sql.Timestamp
import org.apache.spark.sql.catalyst.analysis.TypeCoercion._
import org.apache.spark.sql.catalyst.dsl.expressions._
@@ -1101,34 +1101,13 @@ class TypeCoercionSuite extends AnalysisTest {
test("binary comparison with string promotion") {
ruleTest(PromoteStrings,
GreaterThan(Literal("123"), Literal(1)),
- GreaterThan(Cast(Literal("123"), DoubleType), Cast(Literal(1), DoubleType)))
+ GreaterThan(Cast(Literal("123"), IntegerType), Literal(1)))
ruleTest(PromoteStrings,
LessThan(Literal(true), Literal("123")),
LessThan(Literal(true), Cast(Literal("123"), BooleanType)))
ruleTest(PromoteStrings,
EqualTo(Literal(Array(1, 2)), Literal("123")),
EqualTo(Literal(Array(1, 2)), Literal("123")))
- ruleTest(PromoteStrings,
- GreaterThan(Literal("123"), Literal(1L)),
- GreaterThan(Cast(Literal("123"), DoubleType), Cast(Literal(1L), DoubleType)))
- ruleTest(PromoteStrings,
- GreaterThan(Literal("123"), Literal(0.1)),
- GreaterThan(Cast(Literal("123"), DoubleType), Literal(0.1)))
-
- val date1 = Date.valueOf("2017-07-21")
- val timestamp1 = Timestamp.valueOf("2017-07-21 23:42:12.123")
- ruleTest(PromoteStrings,
- GreaterThan(Literal(date1), Literal("2017-07-01")),
- GreaterThan(Literal(date1), Cast(Literal("2017-07-01"), DateType)))
- ruleTest(PromoteStrings,
- GreaterThan(Literal(timestamp1), Literal("2017-07-01")),
- GreaterThan(Literal(timestamp1), Cast(Literal("2017-07-01"), TimestampType)))
- ruleTest(PromoteStrings,
- GreaterThan(Literal(timestamp1), Cast(Literal("2017-07-01"), DateType)),
- GreaterThan(Literal(timestamp1), Cast(Cast(Literal("2017-07-01"), DateType), TimestampType)))
- ruleTest(PromoteStrings,
- GreaterThan(Literal(timestamp1), Literal(1L)),
- GreaterThan(Cast(Literal(timestamp1), DoubleType), Cast(Literal(1L), DoubleType)))
}
test("cast WindowFrame boundaries to the type they operate upon") {
diff --git a/sql/core/src/test/resources/sql-tests/results/predicate-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/predicate-functions.sql.out
index 5d91be3e72913..8e7e04c8e1c4f 100644
--- a/sql/core/src/test/resources/sql-tests/results/predicate-functions.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/predicate-functions.sql.out
@@ -13,7 +13,7 @@ true
-- !query 1
select 1 = '1'
-- !query 1 schema
-struct<(CAST(1 AS DOUBLE) = CAST(1 AS DOUBLE)):boolean>
+struct<(1 = CAST(1 AS INT)):boolean>
-- !query 1 output
true
@@ -21,7 +21,7 @@ true
-- !query 2
select 1.0 = '1'
-- !query 2 schema
-struct<(CAST(1.0 AS DOUBLE) = CAST(1 AS DOUBLE)):boolean>
+struct<(1.0 = CAST(1 AS DECIMAL(2,1))):boolean>
-- !query 2 output
true
@@ -29,7 +29,7 @@ true
-- !query 3
select 1 > '1'
-- !query 3 schema
-struct<(CAST(1 AS DOUBLE) > CAST(1 AS DOUBLE)):boolean>
+struct<(1 > CAST(1 AS INT)):boolean>
-- !query 3 output
false
@@ -37,7 +37,7 @@ false
-- !query 4
select 2 > '1.0'
-- !query 4 schema
-struct<(CAST(2 AS DOUBLE) > CAST(1.0 AS DOUBLE)):boolean>
+struct<(2 > CAST(1.0 AS INT)):boolean>
-- !query 4 output
true
@@ -45,7 +45,7 @@ true
-- !query 5
select 2 > '2.0'
-- !query 5 schema
-struct<(CAST(2 AS DOUBLE) > CAST(2.0 AS DOUBLE)):boolean>
+struct<(2 > CAST(2.0 AS INT)):boolean>
-- !query 5 output
false
@@ -53,7 +53,7 @@ false
-- !query 6
select 2 > '2.2'
-- !query 6 schema
-struct<(CAST(2 AS DOUBLE) > CAST(2.2 AS DOUBLE)):boolean>
+struct<(2 > CAST(2.2 AS INT)):boolean>
-- !query 6 output
false
@@ -69,7 +69,7 @@ false
-- !query 8
select to_date('2009-07-30 04:17:52') > '2009-07-30 04:17:52'
-- !query 8 schema
-struct<(to_date('2009-07-30 04:17:52') > CAST(2009-07-30 04:17:52 AS DATE)):boolean>
+struct<(CAST(to_date('2009-07-30 04:17:52') AS STRING) > 2009-07-30 04:17:52):boolean>
-- !query 8 output
false
@@ -77,7 +77,7 @@ false
-- !query 9
select 1 >= '1'
-- !query 9 schema
-struct<(CAST(1 AS DOUBLE) >= CAST(1 AS DOUBLE)):boolean>
+struct<(1 >= CAST(1 AS INT)):boolean>
-- !query 9 output
true
@@ -85,7 +85,7 @@ true
-- !query 10
select 2 >= '1.0'
-- !query 10 schema
-struct<(CAST(2 AS DOUBLE) >= CAST(1.0 AS DOUBLE)):boolean>
+struct<(2 >= CAST(1.0 AS INT)):boolean>
-- !query 10 output
true
@@ -93,7 +93,7 @@ true
-- !query 11
select 2 >= '2.0'
-- !query 11 schema
-struct<(CAST(2 AS DOUBLE) >= CAST(2.0 AS DOUBLE)):boolean>
+struct<(2 >= CAST(2.0 AS INT)):boolean>
-- !query 11 output
true
@@ -101,7 +101,7 @@ true
-- !query 12
select 2.0 >= '2.2'
-- !query 12 schema
-struct<(CAST(2.0 AS DOUBLE) >= CAST(2.2 AS DOUBLE)):boolean>
+struct<(2.0 >= CAST(2.2 AS DECIMAL(2,1))):boolean>
-- !query 12 output
false
@@ -117,15 +117,15 @@ true
-- !query 14
select to_date('2009-07-30 04:17:52') >= '2009-07-30 04:17:52'
-- !query 14 schema
-struct<(to_date('2009-07-30 04:17:52') >= CAST(2009-07-30 04:17:52 AS DATE)):boolean>
+struct<(CAST(to_date('2009-07-30 04:17:52') AS STRING) >= 2009-07-30 04:17:52):boolean>
-- !query 14 output
-true
+false
-- !query 15
select 1 < '1'
-- !query 15 schema
-struct<(CAST(1 AS DOUBLE) < CAST(1 AS DOUBLE)):boolean>
+struct<(1 < CAST(1 AS INT)):boolean>
-- !query 15 output
false
@@ -133,7 +133,7 @@ false
-- !query 16
select 2 < '1.0'
-- !query 16 schema
-struct<(CAST(2 AS DOUBLE) < CAST(1.0 AS DOUBLE)):boolean>
+struct<(2 < CAST(1.0 AS INT)):boolean>
-- !query 16 output
false
@@ -141,7 +141,7 @@ false
-- !query 17
select 2 < '2.0'
-- !query 17 schema
-struct<(CAST(2 AS DOUBLE) < CAST(2.0 AS DOUBLE)):boolean>
+struct<(2 < CAST(2.0 AS INT)):boolean>
-- !query 17 output
false
@@ -149,7 +149,7 @@ false
-- !query 18
select 2.0 < '2.2'
-- !query 18 schema
-struct<(CAST(2.0 AS DOUBLE) < CAST(2.2 AS DOUBLE)):boolean>
+struct<(2.0 < CAST(2.2 AS DECIMAL(2,1))):boolean>
-- !query 18 output
true
@@ -165,15 +165,15 @@ false
-- !query 20
select to_date('2009-07-30 04:17:52') < '2009-07-30 04:17:52'
-- !query 20 schema
-struct<(to_date('2009-07-30 04:17:52') < CAST(2009-07-30 04:17:52 AS DATE)):boolean>
+struct<(CAST(to_date('2009-07-30 04:17:52') AS STRING) < 2009-07-30 04:17:52):boolean>
-- !query 20 output
-false
+true
-- !query 21
select 1 <= '1'
-- !query 21 schema
-struct<(CAST(1 AS DOUBLE) <= CAST(1 AS DOUBLE)):boolean>
+struct<(1 <= CAST(1 AS INT)):boolean>
-- !query 21 output
true
@@ -181,7 +181,7 @@ true
-- !query 22
select 2 <= '1.0'
-- !query 22 schema
-struct<(CAST(2 AS DOUBLE) <= CAST(1.0 AS DOUBLE)):boolean>
+struct<(2 <= CAST(1.0 AS INT)):boolean>
-- !query 22 output
false
@@ -189,7 +189,7 @@ false
-- !query 23
select 2 <= '2.0'
-- !query 23 schema
-struct<(CAST(2 AS DOUBLE) <= CAST(2.0 AS DOUBLE)):boolean>
+struct<(2 <= CAST(2.0 AS INT)):boolean>
-- !query 23 output
true
@@ -197,7 +197,7 @@ true
-- !query 24
select 2.0 <= '2.2'
-- !query 24 schema
-struct<(CAST(2.0 AS DOUBLE) <= CAST(2.2 AS DOUBLE)):boolean>
+struct<(2.0 <= CAST(2.2 AS DECIMAL(2,1))):boolean>
-- !query 24 output
true
@@ -213,6 +213,6 @@ true
-- !query 26
select to_date('2009-07-30 04:17:52') <= '2009-07-30 04:17:52'
-- !query 26 schema
-struct<(to_date('2009-07-30 04:17:52') <= CAST(2009-07-30 04:17:52 AS DATE)):boolean>
+struct<(CAST(to_date('2009-07-30 04:17:52') AS STRING) <= 2009-07-30 04:17:52):boolean>
-- !query 26 output
true
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
index 6f01bc40259cb..13341645e8ff8 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
@@ -1966,7 +1966,7 @@ class DataFrameSuite extends QueryTest with SharedSQLContext {
test("SPARK-17913: compare long and string type column may return confusing result") {
val df = Seq(123L -> "123", 19157170390056973L -> "19157170390056971").toDF("i", "j")
- checkAnswer(df.select($"i" === $"j"), Row(true) :: Row(true) :: Nil)
+ checkAnswer(df.select($"i" === $"j"), Row(true) :: Row(false) :: Nil)
}
test("SPARK-19691 Calculating percentile of decimal column fails with ClassCastException") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
index 137ca113be667..c9d6a431b1773 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
@@ -2684,14 +2684,14 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
val str2 = Int.MaxValue.toString + "1"
val str3 = "10"
Seq(str1, str2, str3).toDF("c1").createOrReplaceTempView("v")
- withSQLConf(SQLConf.AUTO_TYPE_CASTING_COMPATIBILITY.key -> "true") {
+ withSQLConf(SQLConf.typeCoercionMode.key -> "hive") {
checkAnswer(sql("SELECT c1 from v where c1 > 0"),
Row(str1) :: Row(str2) :: Row(str3) :: Nil)
checkAnswer(sql("SELECT c1 from v where c1 > 0L"),
Row(str1) :: Row(str2) :: Row(str3) :: Nil)
}
- withSQLConf(SQLConf.AUTO_TYPE_CASTING_COMPATIBILITY.key -> "false") {
+ withSQLConf(SQLConf.typeCoercionMode.key -> "default") {
checkAnswer(sql("SELECT c1 from v where c1 > 0"), Row(str3) :: Nil)
checkAnswer(sql("SELECT c1 from v where c1 > 0L"), Row(str2) :: Row(str3) :: Nil)
}
@@ -2701,7 +2701,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
test("SPARK-21646: CommonTypeForBinaryComparison: DoubleType vs IntegerType") {
withTempView("v") {
Seq(("0", 1), ("-0.4", 2), ("0.6", 3)).toDF("c1", "c2").createOrReplaceTempView("v")
- withSQLConf(SQLConf.AUTO_TYPE_CASTING_COMPATIBILITY.key -> "true") {
+ withSQLConf(SQLConf.typeCoercionMode.key -> "hive") {
checkAnswer(sql("SELECT c1 FROM v WHERE c1 = 0"), Seq(Row("0")))
checkAnswer(sql("SELECT c1 FROM v WHERE c1 = 0L"), Seq(Row("0")))
checkAnswer(sql("SELECT c1 FROM v WHERE c1 = 0.0"), Seq(Row("0")))
@@ -2709,7 +2709,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
checkAnswer(sql("SELECT count(*) FROM v WHERE c1 > 0"), Row(1) :: Nil)
}
- withSQLConf(SQLConf.AUTO_TYPE_CASTING_COMPATIBILITY.key -> "false") {
+ withSQLConf(SQLConf.typeCoercionMode.key -> "default") {
checkAnswer(sql("SELECT c1 FROM v WHERE c1 = 0"), Seq(Row("0"), Row("-0.4"), Row("0.6")))
checkAnswer(sql("SELECT c1 FROM v WHERE c1 = 0L"), Seq(Row("0"), Row("-0.4"), Row("0.6")))
checkAnswer(sql("SELECT c1 FROM v WHERE c1 = 0.0"), Seq(Row("0")))
@@ -2724,13 +2724,13 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
val v1 = Date.valueOf("2017-09-22")
val v2 = Date.valueOf("2017-09-09")
Seq(v1, v2).toDF("c1").createTempView("v")
- withSQLConf(SQLConf.AUTO_TYPE_CASTING_COMPATIBILITY.key -> "true") {
+ withSQLConf(SQLConf.typeCoercionMode.key -> "hive") {
checkAnswer(sql("select c1 from v where c1 > '2017-8-1'"), Row(v1) :: Row(v2) :: Nil)
checkAnswer(sql("select c1 from v where c1 > cast('2017-8-1' as date)"),
Row(v1) :: Row(v2) :: Nil)
}
- withSQLConf(SQLConf.AUTO_TYPE_CASTING_COMPATIBILITY.key -> "false") {
+ withSQLConf(SQLConf.typeCoercionMode.key -> "default") {
checkAnswer(sql("select c1 from v where c1 > '2017-8-1'"), Nil)
checkAnswer(sql("select c1 from v where c1 > cast('2017-8-1' as date)"),
Row(v1) :: Row(v2) :: Nil)
@@ -2743,13 +2743,13 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
val v1 = Timestamp.valueOf("2017-07-21 23:42:12.123")
val v2 = Timestamp.valueOf("2017-08-21 23:42:12.123")
Seq(v1, v2).toDF("c1").createTempView("v")
- withSQLConf(SQLConf.AUTO_TYPE_CASTING_COMPATIBILITY.key -> "true") {
+ withSQLConf(SQLConf.typeCoercionMode.key -> "hive") {
checkAnswer(sql("select c1 from v where c1 > '2017-8-1'"), Row(v2) :: Nil)
checkAnswer(sql("select c1 from v where c1 > cast('2017-8-1' as timestamp)"),
Row(v2) :: Nil)
}
- withSQLConf(SQLConf.AUTO_TYPE_CASTING_COMPATIBILITY.key -> "false") {
+ withSQLConf(SQLConf.typeCoercionMode.key -> "default") {
checkAnswer(sql("select c1 from v where c1 > '2017-8-1'"), Nil)
checkAnswer(sql("select c1 from v where c1 > cast('2017-8-1' as timestamp)"),
Row(v2) :: Nil)
@@ -2762,13 +2762,13 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
val v1 = Timestamp.valueOf("2017-07-21 23:42:12.123")
val v2 = Timestamp.valueOf("2017-08-21 23:42:12.123")
Seq(v1, v2).toDF("c1").createTempView("v")
- withSQLConf(SQLConf.AUTO_TYPE_CASTING_COMPATIBILITY.key -> "true") {
+ withSQLConf(SQLConf.typeCoercionMode.key -> "Hive") {
checkAnswer(sql("select c1 from v where c1 > cast('2017-8-1' as date)"), Row(v2) :: Nil)
checkAnswer(sql("select c1 from v where c1 > cast('2017-8-1' as timestamp)"),
Row(v2) :: Nil)
}
- withSQLConf(SQLConf.AUTO_TYPE_CASTING_COMPATIBILITY.key -> "false") {
+ withSQLConf(SQLConf.typeCoercionMode.key -> "Default") {
checkAnswer(sql("select c1 from v where c1 > cast('2017-8-1' as date)"), Row(v2) :: Nil)
checkAnswer(sql("select c1 from v where c1 > cast('2017-8-1' as timestamp)"),
Row(v2) :: Nil)
@@ -2781,7 +2781,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
val v1 = Timestamp.valueOf("2017-07-21 23:42:12.123")
val v2 = Timestamp.valueOf("2017-08-21 23:42:12.123")
Seq(v1, v2).toDF("c1").createTempView("v")
- withSQLConf(SQLConf.AUTO_TYPE_CASTING_COMPATIBILITY.key -> "true") {
+ withSQLConf(SQLConf.typeCoercionMode.key -> "hive") {
checkAnswer(sql("select c1 from v where c1 > 1"), Row(v1) :: Row(v2) :: Nil)
checkAnswer(sql("select c1 from v where c1 > '2017-8-1'"), Row(v2) :: Nil)
checkAnswer(sql("select c1 from v where c1 > '2017-08-01'"), Row(v2) :: Nil)
@@ -2790,20 +2790,29 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
Row(v2) :: Nil)
}
- withSQLConf(SQLConf.AUTO_TYPE_CASTING_COMPATIBILITY.key -> "false") {
+ withSQLConf(SQLConf.typeCoercionMode.key -> "default") {
val e1 = intercept[AnalysisException] {
sql("select * from v where c1 > 1")
}
assert(e1.getMessage.contains("data type mismatch"))
-
checkAnswer(sql("select c1 from v where c1 > '2017-8-1'"), Nil)
checkAnswer(sql("select c1 from v where c1 > '2017-08-01'"), Row(v2) :: Nil)
-
val e2 = intercept[AnalysisException] {
sql("select * from v where c1 > cast(cast('2017-08-01' as timestamp) as double)")
}
assert(e2.getMessage.contains("data type mismatch"))
}
+
+ val e1 = intercept[AnalysisException] {
+ sql("select * from v where c1 > 1")
+ }
+ assert(e1.getMessage.contains("data type mismatch"))
+ checkAnswer(sql("select c1 from v where c1 > '2017-8-1'"), Nil)
+ checkAnswer(sql("select c1 from v where c1 > '2017-08-01'"), Row(v2) :: Nil)
+ val e2 = intercept[AnalysisException] {
+ sql("select * from v where c1 > cast(cast('2017-08-01' as timestamp) as double)")
+ }
+ assert(e2.getMessage.contains("data type mismatch"))
}
}
}
From 53d673f0fd6cdfc06e34147e364c573ada04114c Mon Sep 17 00:00:00 2001
From: Yuming Wang
Date: Sat, 7 Oct 2017 19:43:51 +0800
Subject: [PATCH 09/19] default -> legacy
---
docs/sql-programming-guide.md | 4 +-
.../sql/catalyst/analysis/Analyzer.scala | 2 +-
.../sql/catalyst/analysis/TypeCoercion.scala | 224 +++++++++++++-----
.../apache/spark/sql/internal/SQLConf.scala | 10 +-
.../org/apache/spark/sql/SQLQuerySuite.scala | 140 +----------
.../hive/execution/TypeCoercionSuite.scala | 170 +++++++++++++
6 files changed, 344 insertions(+), 206 deletions(-)
create mode 100644 sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/TypeCoercionSuite.scala
diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md
index 6ada6cf5cdde5..b2a99b7142513 100644
--- a/docs/sql-programming-guide.md
+++ b/docs/sql-programming-guide.md
@@ -1462,9 +1462,9 @@ that these options will be deprecated in future release as more optimizations ar
spark.sql.typeCoercion.mode |
- default |
+ legacy |
- Whether compatible with Hive. Available options are default and hive.
+ The legacy type coercion mode was used in spark prior to 2.3, and so it continues to be the default to avoid breaking behavior. However, it has logical inconsistencies. The hive mode is preferred for most new applications, though it may require additional manual casting.
|
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 1e934d0aa0e51..fc114efa5c1cb 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
@@ -151,7 +151,7 @@ class Analyzer(
TimeWindowing ::
ResolveInlineTables(conf) ::
ResolveTimeZone(conf) ::
- TypeCoercion.typeCoercionRules ++
+ TypeCoercion.rules(conf) ++
extendedResolutionRules : _*),
Batch("Post-Hoc Resolution", Once, postHocResolutionRules: _*),
Batch("View", Once,
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 d6094cf5a5bf0..bb41afd7c7151 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
@@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.aggregate._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
@@ -44,11 +45,9 @@ import org.apache.spark.sql.types._
*/
object TypeCoercion {
- val typeCoercionRules =
+ private val commonTypeCoercionRules =
PropagateTypes ::
- InConversion ::
WidenSetOperationTypes ::
- PromoteStrings ::
DecimalPrecision ::
BooleanEquality ::
FunctionArgumentConversion ::
@@ -62,6 +61,18 @@ object TypeCoercion {
WindowFrameCoercion ::
Nil
+ def rules(conf: SQLConf): List[Rule[LogicalPlan]] = {
+ if (conf.isHiveTypeCoercionMode) {
+ commonTypeCoercionRules :+
+ HiveInConversion :+
+ HivePromoteStrings
+ } else {
+ commonTypeCoercionRules :+
+ InConversion :+
+ PromoteStrings
+ }
+ }
+
// See https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Types.
// The conversion for integral and floating point types have a linear widening hierarchy:
val numericPrecedence =
@@ -115,46 +126,40 @@ object TypeCoercion {
* is a String and the other is not. It also handles when one op is a Date and the
* other is a Timestamp by making the target type to be String.
*/
- private def findCommonTypeForBinaryComparison(
- plan: LogicalPlan,
- l: DataType,
- r: DataType): Option[DataType] =
- if (!plan.conf.isHiveTypeCoercionMode) {
- (l, r) match {
- // We should cast all relative timestamp/date/string comparison into string comparisons
- // This behaves as a user would expect because timestamp strings sort lexicographically.
- // i.e. TimeStamp(2013-01-01 00:00 ...) < "2014" = true
- case (StringType, DateType) => Some(StringType)
- case (DateType, StringType) => Some(StringType)
- case (StringType, TimestampType) => Some(StringType)
- case (TimestampType, StringType) => Some(StringType)
- case (TimestampType, DateType) => Some(StringType)
- case (DateType, TimestampType) => Some(StringType)
- case (StringType, NullType) => Some(StringType)
- case (NullType, StringType) => Some(StringType)
- case (l: StringType, r: AtomicType) if r != StringType => Some(r)
- case (l: AtomicType, r: StringType) if (l != StringType) => Some(l)
- case (l, r) => None
- }
- } else {
- (l, r) match {
- // Follow hive's binary comparison action:
- // https://github.com/apache/hive/blob/rel/storage-release-2.4.0/ql/src/java/
- // org/apache/hadoop/hive/ql/exec/FunctionRegistry.java#L781
- case (StringType, DateType) => Some(DateType)
- case (DateType, StringType) => Some(DateType)
- case (StringType, TimestampType) => Some(TimestampType)
- case (TimestampType, StringType) => Some(TimestampType)
- case (TimestampType, DateType) => Some(TimestampType)
- case (DateType, TimestampType) => Some(TimestampType)
- case (StringType, NullType) => Some(StringType)
- case (NullType, StringType) => Some(StringType)
- case (StringType | TimestampType, r: NumericType) => Some(DoubleType)
- case (l: NumericType, StringType | TimestampType) => Some(DoubleType)
- case (l: StringType, r: AtomicType) if r != StringType => Some(r)
- case (l: AtomicType, r: StringType) if l != StringType => Some(l)
- case _ => None
- }
+ val findCommonTypeForBinaryComparison: (DataType, DataType) => Option[DataType] = {
+ // We should cast all relative timestamp/date/string comparison into string comparisons
+ // This behaves as a user would expect because timestamp strings sort lexicographically.
+ // i.e. TimeStamp(2013-01-01 00:00 ...) < "2014" = true
+ case (StringType, DateType) => Some(StringType)
+ case (DateType, StringType) => Some(StringType)
+ case (StringType, TimestampType) => Some(StringType)
+ case (TimestampType, StringType) => Some(StringType)
+ case (TimestampType, DateType) => Some(StringType)
+ case (DateType, TimestampType) => Some(StringType)
+ case (StringType, NullType) => Some(StringType)
+ case (NullType, StringType) => Some(StringType)
+ case (l: StringType, r: AtomicType) if r != StringType => Some(r)
+ case (l: AtomicType, r: StringType) if (l != StringType) => Some(l)
+ case (l, r) => None
+ }
+
+ val findCommonTypeToCompatibleWithHive: (DataType, DataType) => Option[DataType] = {
+ // Follow hive's binary comparison action:
+ // https://github.com/apache/hive/blob/rel/storage-release-2.4.0/ql/src/java/
+ // org/apache/hadoop/hive/ql/exec/FunctionRegistry.java#L781
+ case (StringType, DateType) => Some(DateType)
+ case (DateType, StringType) => Some(DateType)
+ case (StringType, TimestampType) => Some(TimestampType)
+ case (TimestampType, StringType) => Some(TimestampType)
+ case (TimestampType, DateType) => Some(TimestampType)
+ case (DateType, TimestampType) => Some(TimestampType)
+ case (StringType, NullType) => Some(StringType)
+ case (NullType, StringType) => Some(StringType)
+ case (StringType | TimestampType, r: NumericType) => Some(DoubleType)
+ case (l: NumericType, StringType | TimestampType) => Some(DoubleType)
+ case (l: StringType, r: AtomicType) if r != StringType => Some(r)
+ case (l: AtomicType, r: StringType) if l != StringType => Some(l)
+ case _ => None
}
/**
@@ -350,17 +355,18 @@ object TypeCoercion {
}
}
+ private def castExpr(expr: Expression, targetType: DataType): Expression = {
+ (expr.dataType, targetType) match {
+ case (NullType, dt) => Literal.create(null, targetType)
+ case (l, dt) if (l != dt) => Cast(expr, targetType)
+ case _ => expr
+ }
+ }
+
/**
* Promotes strings that appear in arithmetic expressions.
*/
object PromoteStrings extends Rule[LogicalPlan] {
- private def castExpr(expr: Expression, targetType: DataType): Expression = {
- (expr.dataType, targetType) match {
- case (NullType, dt) => Literal.create(null, targetType)
- case (l, dt) if (l != dt) => Cast(expr, targetType)
- case _ => expr
- }
- }
def apply(plan: LogicalPlan): LogicalPlan = plan resolveExpressions {
// Skip nodes who's children have not been resolved yet.
@@ -378,8 +384,8 @@ object TypeCoercion {
case p @ Equality(left @ TimestampType(), right @ StringType()) =>
p.makeCopy(Array(left, Cast(right, TimestampType)))
case p @ BinaryComparison(left, right)
- if findCommonTypeForBinaryComparison(plan, left.dataType, right.dataType).isDefined =>
- val commonType = findCommonTypeForBinaryComparison(plan, left.dataType, right.dataType).get
+ if findCommonTypeForBinaryComparison(left.dataType, right.dataType).isDefined =>
+ val commonType = findCommonTypeForBinaryComparison(left.dataType, right.dataType).get
p.makeCopy(Array(castExpr(left, commonType), castExpr(right, commonType)))
case Abs(e @ StringType()) => Abs(Cast(e, DoubleType))
@@ -396,6 +402,52 @@ object TypeCoercion {
}
}
+ /**
+ * Promotes strings that appear in arithmetic expressions to compatible with Hive.
+ */
+ object HivePromoteStrings extends Rule[LogicalPlan] {
+
+ def apply(plan: LogicalPlan): LogicalPlan = plan resolveExpressions {
+ // Skip nodes who's children have not been resolved yet.
+ case e if !e.childrenResolved => e
+
+ case a @ BinaryArithmetic(left @ StringType(), right) =>
+ a.makeCopy(Array(Cast(left, DoubleType), right))
+ case a @ BinaryArithmetic(left, right @ StringType()) =>
+ a.makeCopy(Array(left, Cast(right, DoubleType)))
+
+ case p @ Equality(left, right)
+ if findCommonTypeToCompatibleWithHive(left.dataType, right.dataType).isDefined =>
+ val commonType = findCommonTypeToCompatibleWithHive(left.dataType, right.dataType).get
+ p.makeCopy(Array(castExpr(left, commonType), castExpr(right, commonType)))
+ case p @ BinaryComparison(left, right)
+ if findCommonTypeToCompatibleWithHive(left.dataType, right.dataType).isDefined =>
+ val commonType = findCommonTypeToCompatibleWithHive(left.dataType, right.dataType).get
+ p.makeCopy(Array(castExpr(left, commonType), castExpr(right, commonType)))
+
+ case Abs(e @ StringType()) => Abs(Cast(e, DoubleType))
+ case Sum(e @ StringType()) => Sum(Cast(e, DoubleType))
+ case Average(e @ StringType()) => Average(Cast(e, DoubleType))
+ case StddevPop(e @ StringType()) => StddevPop(Cast(e, DoubleType))
+ case StddevSamp(e @ StringType()) => StddevSamp(Cast(e, DoubleType))
+ case UnaryMinus(e @ StringType()) => UnaryMinus(Cast(e, DoubleType))
+ case UnaryPositive(e @ StringType()) => UnaryPositive(Cast(e, DoubleType))
+ case VariancePop(e @ StringType()) => VariancePop(Cast(e, DoubleType))
+ case VarianceSamp(e @ StringType()) => VarianceSamp(Cast(e, DoubleType))
+ case Skewness(e @ StringType()) => Skewness(Cast(e, DoubleType))
+ case Kurtosis(e @ StringType()) => Kurtosis(Cast(e, DoubleType))
+ }
+ }
+
+ private def flattenExpr(expr: Expression): Seq[Expression] = {
+ expr match {
+ // Multi columns in IN clause is represented as a CreateNamedStruct.
+ // flatten the named struct to get the list of expressions.
+ case cns: CreateNamedStruct => cns.valExprs
+ case expr => Seq(expr)
+ }
+ }
+
/**
* Handles type coercion for both IN expression with subquery and IN
* expressions without subquery.
@@ -411,14 +463,6 @@ object TypeCoercion {
* Analysis Exception will be raised at the type checking phase.
*/
object InConversion extends Rule[LogicalPlan] {
- private def flattenExpr(expr: Expression): Seq[Expression] = {
- expr match {
- // Multi columns in IN clause is represented as a CreateNamedStruct.
- // flatten the named struct to get the list of expressions.
- case cns: CreateNamedStruct => cns.valExprs
- case expr => Seq(expr)
- }
- }
def apply(plan: LogicalPlan): LogicalPlan = plan resolveExpressions {
// Skip nodes who's children have not been resolved yet.
@@ -435,7 +479,7 @@ object TypeCoercion {
val rhs = sub.output
val commonTypes = lhs.zip(rhs).flatMap { case (l, r) =>
- findCommonTypeForBinaryComparison(plan, l.dataType, r.dataType)
+ findCommonTypeForBinaryComparison(l.dataType, r.dataType)
.orElse(findTightestCommonType(l.dataType, r.dataType))
}
@@ -473,6 +517,64 @@ object TypeCoercion {
}
}
+ /**
+ * Handles type coercion for IN expression to compatible with Hive.
+ */
+ object HiveInConversion extends Rule[LogicalPlan] {
+
+ def apply(plan: LogicalPlan): LogicalPlan = plan resolveExpressions {
+ // Skip nodes who's children have not been resolved yet.
+ case e if !e.childrenResolved => e
+
+ // Handle type casting required between value expression and subquery output
+ // in IN subquery.
+ case i @ In(a, Seq(ListQuery(sub, children, exprId, _)))
+ if !i.resolved && flattenExpr(a).length == sub.output.length =>
+ // LHS is the value expression of IN subquery.
+ val lhs = flattenExpr(a)
+
+ // RHS is the subquery output.
+ val rhs = sub.output
+
+ val commonTypes = lhs.zip(rhs).flatMap { case (l, r) =>
+ findCommonTypeToCompatibleWithHive(l.dataType, r.dataType)
+ .orElse(findTightestCommonType(l.dataType, r.dataType))
+
+ }
+
+ // The number of columns/expressions must match between LHS and RHS of an
+ // IN subquery expression.
+ if (commonTypes.length == lhs.length) {
+ val castedRhs = rhs.zip(commonTypes).map {
+ case (e, dt) if e.dataType != dt => Alias(Cast(e, dt), e.name)()
+ case (e, _) => e
+ }
+ val castedLhs = lhs.zip(commonTypes).map {
+ case (e, dt) if e.dataType != dt => Cast(e, dt)
+ case (e, _) => e
+ }
+
+ // Before constructing the In expression, wrap the multi values in LHS
+ // in a CreatedNamedStruct.
+ val newLhs = castedLhs match {
+ case Seq(lhs) => lhs
+ case _ => CreateStruct(castedLhs)
+ }
+
+ val newSub = Project(castedRhs, sub)
+ In(newLhs, Seq(ListQuery(newSub, children, exprId, newSub.output)))
+ } else {
+ i
+ }
+
+ case i @ In(a, b) if b.exists(_.dataType != a.dataType) =>
+ findWiderCommonType(i.children.map(_.dataType)) match {
+ case Some(finalDataType) => i.withNewChildren(i.children.map(Cast(_, finalDataType)))
+ case None => i
+ }
+ }
+ }
+
/**
* Changes numeric values to booleans so that expressions like true = 1 can be evaluated.
*/
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
index ee22e1823b516..813494e6dd176 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
@@ -927,11 +927,15 @@ object SQLConf {
val typeCoercionMode =
buildConf("spark.sql.typeCoercion.mode")
- .doc("Whether compatible with Hive.")
+ .doc("The 'legacy' typeCoercion mode was used in spark prior to 2.3, " +
+ "and so it continues to be the default to avoid breaking behavior. " +
+ "However, it has logical inconsistencies. " +
+ "The 'hive' mode is preferred for most new applications, " +
+ "though it may require additional manual casting.")
.stringConf
.transform(_.toLowerCase(Locale.ROOT))
- .checkValues(Set("default", "hive"))
- .createWithDefault("default")
+ .checkValues(Set("legacy", "hive"))
+ .createWithDefault("legacy")
object Deprecated {
val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks"
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
index c9d6a431b1773..93a7777b70b46 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql
import java.io.File
import java.math.MathContext
import java.net.{MalformedURLException, URL}
-import java.sql.{Date, Timestamp}
+import java.sql.Timestamp
import java.util.concurrent.atomic.AtomicBoolean
import org.apache.spark.{AccumulatorSuite, SparkException}
@@ -2677,142 +2677,4 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
checkAnswer(df, Row(1, 1, 1))
}
}
-
- test("SPARK-21646: CommonTypeForBinaryComparison: StringType vs NumericType") {
- withTempView("v") {
- val str1 = Long.MaxValue.toString + "1"
- val str2 = Int.MaxValue.toString + "1"
- val str3 = "10"
- Seq(str1, str2, str3).toDF("c1").createOrReplaceTempView("v")
- withSQLConf(SQLConf.typeCoercionMode.key -> "hive") {
- checkAnswer(sql("SELECT c1 from v where c1 > 0"),
- Row(str1) :: Row(str2) :: Row(str3) :: Nil)
- checkAnswer(sql("SELECT c1 from v where c1 > 0L"),
- Row(str1) :: Row(str2) :: Row(str3) :: Nil)
- }
-
- withSQLConf(SQLConf.typeCoercionMode.key -> "default") {
- checkAnswer(sql("SELECT c1 from v where c1 > 0"), Row(str3) :: Nil)
- checkAnswer(sql("SELECT c1 from v where c1 > 0L"), Row(str2) :: Row(str3) :: Nil)
- }
- }
- }
-
- test("SPARK-21646: CommonTypeForBinaryComparison: DoubleType vs IntegerType") {
- withTempView("v") {
- Seq(("0", 1), ("-0.4", 2), ("0.6", 3)).toDF("c1", "c2").createOrReplaceTempView("v")
- withSQLConf(SQLConf.typeCoercionMode.key -> "hive") {
- checkAnswer(sql("SELECT c1 FROM v WHERE c1 = 0"), Seq(Row("0")))
- checkAnswer(sql("SELECT c1 FROM v WHERE c1 = 0L"), Seq(Row("0")))
- checkAnswer(sql("SELECT c1 FROM v WHERE c1 = 0.0"), Seq(Row("0")))
- checkAnswer(sql("SELECT c1 FROM v WHERE c1 = -0.4"), Seq(Row("-0.4")))
- checkAnswer(sql("SELECT count(*) FROM v WHERE c1 > 0"), Row(1) :: Nil)
- }
-
- withSQLConf(SQLConf.typeCoercionMode.key -> "default") {
- checkAnswer(sql("SELECT c1 FROM v WHERE c1 = 0"), Seq(Row("0"), Row("-0.4"), Row("0.6")))
- checkAnswer(sql("SELECT c1 FROM v WHERE c1 = 0L"), Seq(Row("0"), Row("-0.4"), Row("0.6")))
- checkAnswer(sql("SELECT c1 FROM v WHERE c1 = 0.0"), Seq(Row("0")))
- checkAnswer(sql("SELECT c1 FROM v WHERE c1 = -0.4"), Seq(Row("-0.4")))
- checkAnswer(sql("SELECT count(*) FROM v WHERE c1 > 0"), Row(0) :: Nil)
- }
- }
- }
-
- test("SPARK-21646: CommonTypeForBinaryComparison: StringType vs DateType") {
- withTempView("v") {
- val v1 = Date.valueOf("2017-09-22")
- val v2 = Date.valueOf("2017-09-09")
- Seq(v1, v2).toDF("c1").createTempView("v")
- withSQLConf(SQLConf.typeCoercionMode.key -> "hive") {
- checkAnswer(sql("select c1 from v where c1 > '2017-8-1'"), Row(v1) :: Row(v2) :: Nil)
- checkAnswer(sql("select c1 from v where c1 > cast('2017-8-1' as date)"),
- Row(v1) :: Row(v2) :: Nil)
- }
-
- withSQLConf(SQLConf.typeCoercionMode.key -> "default") {
- checkAnswer(sql("select c1 from v where c1 > '2017-8-1'"), Nil)
- checkAnswer(sql("select c1 from v where c1 > cast('2017-8-1' as date)"),
- Row(v1) :: Row(v2) :: Nil)
- }
- }
- }
-
- test("SPARK-21646: CommonTypeForBinaryComparison: StringType vs TimestampType") {
- withTempView("v") {
- val v1 = Timestamp.valueOf("2017-07-21 23:42:12.123")
- val v2 = Timestamp.valueOf("2017-08-21 23:42:12.123")
- Seq(v1, v2).toDF("c1").createTempView("v")
- withSQLConf(SQLConf.typeCoercionMode.key -> "hive") {
- checkAnswer(sql("select c1 from v where c1 > '2017-8-1'"), Row(v2) :: Nil)
- checkAnswer(sql("select c1 from v where c1 > cast('2017-8-1' as timestamp)"),
- Row(v2) :: Nil)
- }
-
- withSQLConf(SQLConf.typeCoercionMode.key -> "default") {
- checkAnswer(sql("select c1 from v where c1 > '2017-8-1'"), Nil)
- checkAnswer(sql("select c1 from v where c1 > cast('2017-8-1' as timestamp)"),
- Row(v2) :: Nil)
- }
- }
- }
-
- test("SPARK-21646: CommonTypeForBinaryComparison: TimestampType vs DateType") {
- withTempView("v") {
- val v1 = Timestamp.valueOf("2017-07-21 23:42:12.123")
- val v2 = Timestamp.valueOf("2017-08-21 23:42:12.123")
- Seq(v1, v2).toDF("c1").createTempView("v")
- withSQLConf(SQLConf.typeCoercionMode.key -> "Hive") {
- checkAnswer(sql("select c1 from v where c1 > cast('2017-8-1' as date)"), Row(v2) :: Nil)
- checkAnswer(sql("select c1 from v where c1 > cast('2017-8-1' as timestamp)"),
- Row(v2) :: Nil)
- }
-
- withSQLConf(SQLConf.typeCoercionMode.key -> "Default") {
- checkAnswer(sql("select c1 from v where c1 > cast('2017-8-1' as date)"), Row(v2) :: Nil)
- checkAnswer(sql("select c1 from v where c1 > cast('2017-8-1' as timestamp)"),
- Row(v2) :: Nil)
- }
- }
- }
-
- test("SPARK-21646: CommonTypeForBinaryComparison: TimestampType vs NumericType") {
- withTempView("v") {
- val v1 = Timestamp.valueOf("2017-07-21 23:42:12.123")
- val v2 = Timestamp.valueOf("2017-08-21 23:42:12.123")
- Seq(v1, v2).toDF("c1").createTempView("v")
- withSQLConf(SQLConf.typeCoercionMode.key -> "hive") {
- checkAnswer(sql("select c1 from v where c1 > 1"), Row(v1) :: Row(v2) :: Nil)
- checkAnswer(sql("select c1 from v where c1 > '2017-8-1'"), Row(v2) :: Nil)
- checkAnswer(sql("select c1 from v where c1 > '2017-08-01'"), Row(v2) :: Nil)
- checkAnswer(
- sql("select * from v where c1 > cast(cast('2017-08-01' as timestamp) as double)"),
- Row(v2) :: Nil)
- }
-
- withSQLConf(SQLConf.typeCoercionMode.key -> "default") {
- val e1 = intercept[AnalysisException] {
- sql("select * from v where c1 > 1")
- }
- assert(e1.getMessage.contains("data type mismatch"))
- checkAnswer(sql("select c1 from v where c1 > '2017-8-1'"), Nil)
- checkAnswer(sql("select c1 from v where c1 > '2017-08-01'"), Row(v2) :: Nil)
- val e2 = intercept[AnalysisException] {
- sql("select * from v where c1 > cast(cast('2017-08-01' as timestamp) as double)")
- }
- assert(e2.getMessage.contains("data type mismatch"))
- }
-
- val e1 = intercept[AnalysisException] {
- sql("select * from v where c1 > 1")
- }
- assert(e1.getMessage.contains("data type mismatch"))
- checkAnswer(sql("select c1 from v where c1 > '2017-8-1'"), Nil)
- checkAnswer(sql("select c1 from v where c1 > '2017-08-01'"), Row(v2) :: Nil)
- val e2 = intercept[AnalysisException] {
- sql("select * from v where c1 > cast(cast('2017-08-01' as timestamp) as double)")
- }
- assert(e2.getMessage.contains("data type mismatch"))
- }
- }
}
diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/TypeCoercionSuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/TypeCoercionSuite.scala
new file mode 100644
index 0000000000000..68aeecedbbc50
--- /dev/null
+++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/TypeCoercionSuite.scala
@@ -0,0 +1,170 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.hive.execution
+
+import java.sql.{Date, Timestamp}
+
+import org.apache.spark.SparkConf
+import org.apache.spark.sql._
+import org.apache.spark.sql.LocalSparkSession.withSparkSession
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SharedSQLContext
+
+class TypeCoercionSuite extends QueryTest with SharedSQLContext {
+
+ val hiveSparkConf = new SparkConf()
+ .setMaster("local")
+ .set(SQLConf.typeCoercionMode.key, "hive")
+
+ val legacySparkConf = new SparkConf()
+ .setMaster("local")
+ .set(SQLConf.typeCoercionMode.key, "legacy")
+
+ test("SPARK-21646: CommonTypeForBinaryComparison: StringType vs NumericType") {
+ val str1 = Long.MaxValue.toString + "1"
+ val str2 = Int.MaxValue.toString + "1"
+ val str3 = "10"
+ val str4 = "0"
+ val str5 = "-0.4"
+ val str6 = "0.6"
+
+ withSparkSession(SparkSession.builder.config(hiveSparkConf).getOrCreate()) { spark =>
+ import spark.implicits._
+ Seq(str1, str2, str3, str4, str5, str6).toDF("c1").createOrReplaceTempView("v")
+ checkAnswer(spark.sql("SELECT c1 from v where c1 > 0"),
+ Row(str1) :: Row(str2) :: Row(str3) :: Row(str6) :: Nil)
+ checkAnswer(spark.sql("SELECT c1 from v where c1 > 0L"),
+ Row(str1) :: Row(str2) :: Row(str3) :: Row(str6) :: Nil)
+ checkAnswer(spark.sql("SELECT c1 FROM v WHERE c1 = 0"),
+ Seq(Row("0")))
+ }
+
+ withSparkSession(SparkSession.builder.config(legacySparkConf).getOrCreate()) { spark =>
+ import spark.implicits._
+ Seq(str1, str2, str3, str4, str5, str6).toDF("c1").createOrReplaceTempView("v")
+ checkAnswer(spark.sql("SELECT c1 from v where c1 > 0"),
+ Row(str3) :: Nil)
+ checkAnswer(spark.sql("SELECT c1 from v where c1 > 0L"),
+ Row(str2) :: Row(str3) :: Nil)
+ checkAnswer(spark.sql("SELECT c1 FROM v WHERE c1 = 0"),
+ Seq(Row("0"), Row("-0.4"), Row("0.6")))
+ }
+ }
+
+ test("SPARK-21646: CommonTypeForBinaryComparison: StringType vs DateType") {
+ val v1 = Date.valueOf("2017-09-22")
+ val v2 = Date.valueOf("2017-09-09")
+
+ withSparkSession(SparkSession.builder.config(hiveSparkConf).getOrCreate()) { spark =>
+ import spark.implicits._
+ Seq(v1, v2).toDF("c1").createTempView("v")
+ checkAnswer(spark.sql("select c1 from v where c1 > '2017-8-1'"), Row(v1) :: Row(v2) :: Nil)
+ checkAnswer(spark.sql("select c1 from v where c1 > '2014'"), Row(v1) :: Row(v2) :: Nil)
+ checkAnswer(spark.sql("select c1 from v where c1 > cast('2017-8-1' as date)"),
+ Row(v1) :: Row(v2) :: Nil)
+ }
+
+ withSparkSession(SparkSession.builder.config(legacySparkConf).getOrCreate()) { spark =>
+ import spark.implicits._
+ Seq(v1, v2).toDF("c1").createTempView("v")
+ checkAnswer(spark.sql("select c1 from v where c1 > '2017-8-1'"), Nil)
+ checkAnswer(spark.sql("select c1 from v where c1 > '2014'"), Row(v1) :: Row(v2) :: Nil)
+ checkAnswer(spark.sql("select c1 from v where c1 > cast('2017-8-1' as date)"),
+ Row(v1) :: Row(v2) :: Nil)
+ }
+ }
+
+ test("SPARK-21646: CommonTypeForBinaryComparison: StringType vs TimestampType") {
+ val v1 = Timestamp.valueOf("2017-07-21 23:42:12.123")
+ val v2 = Timestamp.valueOf("2017-08-21 23:42:12.123")
+ val v3 = Timestamp.valueOf("2017-08-21 23:42:12.0")
+
+ withSparkSession(SparkSession.builder.config(hiveSparkConf).getOrCreate()) { spark =>
+ import spark.implicits._
+ Seq(v1, v2, v3).toDF("c1").createTempView("v")
+ checkAnswer(spark.sql("select c1 from v where c1 > '2017-8-1'"), Row(v2) :: Row(v3) :: Nil)
+ checkAnswer(spark.sql("select c1 from v where c1 > '2017-08-21 23:42:12'"),
+ Row(v2) :: Nil)
+ checkAnswer(spark.sql("select c1 from v where c1 > cast('2017-8-1' as timestamp)"),
+ Row(v2) :: Row(v3) :: Nil)
+ }
+
+ withSparkSession(SparkSession.builder.config(legacySparkConf).getOrCreate()) { spark =>
+ import spark.implicits._
+ Seq(v1, v2, v3).toDF("c1").createTempView("v")
+ checkAnswer(spark.sql("select c1 from v where c1 > '2017-8-1'"), Nil)
+ checkAnswer(spark.sql("select c1 from v where c1 >= '2017-08-21 23:42:12'"),
+ Row(v2) :: Row(v3) :: Nil)
+ checkAnswer(spark.sql("select c1 from v where c1 > cast('2017-8-1' as timestamp)"),
+ Row(v2) :: Row(v3) :: Nil)
+ }
+ }
+
+ test("SPARK-21646: CommonTypeForBinaryComparison: TimestampType vs DateType") {
+ val v1 = Timestamp.valueOf("2017-07-21 23:42:12.123")
+ val v2 = Timestamp.valueOf("2017-08-21 23:42:12.123")
+
+ withSparkSession(SparkSession.builder.config(hiveSparkConf).getOrCreate()) { spark =>
+ import spark.implicits._
+ Seq(v1, v2).toDF("c1").createTempView("v")
+ checkAnswer(spark.sql("select c1 from v where c1 > cast('2017-8-1' as date)"), Row(v2) :: Nil)
+ checkAnswer(spark.sql("select c1 from v where c1 > cast('2017-8-1' as timestamp)"),
+ Row(v2) :: Nil)
+ }
+
+ withSparkSession(SparkSession.builder.config(legacySparkConf).getOrCreate()) { spark =>
+ import spark.implicits._
+ Seq(v1, v2).toDF("c1").createTempView("v")
+ checkAnswer(spark.sql("select c1 from v where c1 > cast('2017-8-1' as date)"), Row(v2) :: Nil)
+ checkAnswer(spark.sql("select c1 from v where c1 > cast('2017-8-1' as timestamp)"),
+ Row(v2) :: Nil)
+ }
+ }
+
+ test("SPARK-21646: CommonTypeForBinaryComparison: TimestampType vs NumericType") {
+ val v1 = Timestamp.valueOf("2017-07-21 23:42:12.123")
+ val v2 = Timestamp.valueOf("2017-08-21 23:42:12.123")
+
+ withSparkSession(SparkSession.builder.config(hiveSparkConf).getOrCreate()) { spark =>
+ import spark.implicits._
+ Seq(v1, v2).toDF("c1").createTempView("v")
+ checkAnswer(spark.sql("select c1 from v where c1 > 1"), Row(v1) :: Row(v2) :: Nil)
+ checkAnswer(spark.sql("select c1 from v where c1 > '2017-8-1'"), Row(v2) :: Nil)
+ checkAnswer(spark.sql("select c1 from v where c1 > '2017-08-01'"), Row(v2) :: Nil)
+ checkAnswer(
+ spark.sql("select * from v where c1 > cast(cast('2017-08-01' as timestamp) as double)"),
+ Row(v2) :: Nil)
+ }
+
+ withSparkSession(SparkSession.builder.config(legacySparkConf).getOrCreate()) { spark =>
+ import spark.implicits._
+ Seq(v1, v2).toDF("c1").createTempView("v")
+ val e1 = intercept[AnalysisException] {
+ spark.sql("select * from v where c1 > 1")
+ }
+ assert(e1.getMessage.contains("data type mismatch"))
+ checkAnswer(spark.sql("select c1 from v where c1 > '2017-8-1'"), Nil)
+ checkAnswer(spark.sql("select c1 from v where c1 > '2017-08-01'"), Row(v2) :: Nil)
+ val e2 = intercept[AnalysisException] {
+ spark.sql("select * from v where c1 > cast(cast('2017-08-01' as timestamp) as double)")
+ }
+ assert(e2.getMessage.contains("data type mismatch"))
+ }
+ }
+
+}
From 8da0cf07416372850fac302b42bebebbcfd88ff4 Mon Sep 17 00:00:00 2001
From: Yuming Wang
Date: Sun, 8 Oct 2017 00:31:23 +0800
Subject: [PATCH 10/19] Fix test error
---
.../apache/spark/sql/hive/execution/TypeCoercionSuite.scala | 5 +++--
1 file changed, 3 insertions(+), 2 deletions(-)
diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/TypeCoercionSuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/TypeCoercionSuite.scala
index 68aeecedbbc50..748baf31b905f 100644
--- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/TypeCoercionSuite.scala
+++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/TypeCoercionSuite.scala
@@ -23,9 +23,10 @@ import org.apache.spark.SparkConf
import org.apache.spark.sql._
import org.apache.spark.sql.LocalSparkSession.withSparkSession
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
-class TypeCoercionSuite extends QueryTest with SharedSQLContext {
+class TypeCoercionSuite extends QueryTest {
+
+ val spark: SparkSession = null
val hiveSparkConf = new SparkConf()
.setMaster("local")
From 2ada11a36d9fe9aa0f1048b535ac2da2a129c49f Mon Sep 17 00:00:00 2001
From: Yuming Wang
Date: Mon, 9 Oct 2017 23:54:19 +0800
Subject: [PATCH 11/19] Refactor TypeCoercionModeSuite
---
.../sql/execution/TypeCoercionModeSuite.scala | 212 ++++++++++++++++++
.../hive/execution/TypeCoercionSuite.scala | 171 --------------
2 files changed, 212 insertions(+), 171 deletions(-)
create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/TypeCoercionModeSuite.scala
delete mode 100644 sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/TypeCoercionSuite.scala
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/TypeCoercionModeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/TypeCoercionModeSuite.scala
new file mode 100644
index 0000000000000..b71813111121c
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/TypeCoercionModeSuite.scala
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution
+
+import java.sql.{Date, Timestamp}
+
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.spark.{SparkConf, SparkFunSuite}
+import org.apache.spark.sql._
+import org.apache.spark.sql.internal.SQLConf
+
+class TypeCoercionModeSuite extends SparkFunSuite with BeforeAndAfterAll {
+
+ private var originalActiveSparkSession: Option[SparkSession] = _
+ private var originalInstantiatedSparkSession: Option[SparkSession] = _
+
+ override protected def beforeAll(): Unit = {
+ originalActiveSparkSession = SparkSession.getActiveSession
+ originalInstantiatedSparkSession = SparkSession.getDefaultSession
+
+ SparkSession.clearActiveSession()
+ SparkSession.clearDefaultSession()
+ }
+
+ override protected def afterAll(): Unit = {
+ originalActiveSparkSession.foreach(ctx => SparkSession.setActiveSession(ctx))
+ originalInstantiatedSparkSession.foreach(ctx => SparkSession.setDefaultSession(ctx))
+ }
+
+ var sparkSession: SparkSession = _
+
+ def withTypeCoercionMode[T](typeCoercionMode: String)(f: SparkSession => T): T = {
+ try {
+ val sparkConf = new SparkConf(false)
+ .setMaster("local")
+ .setAppName(this.getClass.getName)
+ .set("spark.ui.enabled", "false")
+ .set("spark.driver.allowMultipleContexts", "true")
+ .set(SQLConf.typeCoercionMode.key, typeCoercionMode)
+
+ sparkSession = SparkSession.builder().config(sparkConf).getOrCreate()
+ f(sparkSession)
+ } finally {
+ if (sparkSession != null) {
+ sparkSession.sql("DROP TABLE IF EXISTS v")
+ sparkSession.stop()
+ }
+ }
+ }
+
+ test("CommonTypeForBinaryComparison: StringType vs NumericType") {
+ val str1 = Long.MaxValue.toString + "1"
+ val str2 = Int.MaxValue.toString + "1"
+ val str3 = "10"
+ val str4 = "0"
+ val str5 = "-0.4"
+ val str6 = "0.6"
+
+ val data = Seq(str1, str2, str3, str4, str5, str6)
+
+ val q1 = "SELECT c1 from v where c1 > 0"
+ val q2 = "SELECT c1 from v where c1 > 0L"
+ val q3 = "SELECT c1 FROM v WHERE c1 = 0"
+
+ withTypeCoercionMode("hive") { spark =>
+ import spark.implicits._
+ data.toDF("c1").createOrReplaceTempView("v")
+ QueryTest.checkAnswer(spark.sql(q1), Row(str1) :: Row(str2) :: Row(str3) :: Row(str6) :: Nil)
+ QueryTest.checkAnswer(spark.sql(q2), Row(str1) :: Row(str2) :: Row(str3) :: Row(str6) :: Nil)
+ QueryTest.checkAnswer(spark.sql(q3), Seq(Row(str4)))
+ }
+
+ withTypeCoercionMode("legacy") { spark =>
+ import spark.implicits._
+ data.toDF("c1").createOrReplaceTempView("v")
+ QueryTest.checkAnswer(spark.sql(q1), Row(str3) :: Nil)
+ QueryTest.checkAnswer(spark.sql(q2), Row(str2) :: Row(str3) :: Nil)
+ QueryTest.checkAnswer(spark.sql(q3), Row(str4) :: Row(str5) :: Row(str6) :: Nil)
+ }
+ }
+
+ test("CommonTypeForBinaryComparison: StringType vs DateType") {
+ val v1 = Date.valueOf("2017-09-22")
+ val v2 = Date.valueOf("2017-09-09")
+
+ val data = Seq(v1, v2)
+
+ val q1 = "select c1 from v where c1 > '2017-8-1'"
+ val q2 = "select c1 from v where c1 > '2014'"
+ val q3 = "select c1 from v where c1 > cast('2017-8-1' as date)"
+
+ withTypeCoercionMode("hive") { spark =>
+ import spark.implicits._
+ data.toDF("c1").createTempView("v")
+ QueryTest.checkAnswer(spark.sql(q1), Row(v1) :: Row(v2) :: Nil)
+ QueryTest.checkAnswer(spark.sql(q2), Row(v1) :: Row(v2) :: Nil)
+ QueryTest.checkAnswer(spark.sql(q3), Row(v1) :: Row(v2) :: Nil)
+ }
+
+ withTypeCoercionMode("legacy") { spark =>
+ import spark.implicits._
+ data.toDF("c1").createTempView("v")
+ QueryTest.checkAnswer(spark.sql(q1), Nil)
+ QueryTest.checkAnswer(spark.sql(q2), Row(v1) :: Row(v2) :: Nil)
+ QueryTest.checkAnswer(spark.sql(q3), Row(v1) :: Row(v2) :: Nil)
+ }
+ }
+
+ test("CommonTypeForBinaryComparison: StringType vs TimestampType") {
+ val v1 = Timestamp.valueOf("2017-07-21 23:42:12.123")
+ val v2 = Timestamp.valueOf("2017-08-21 23:42:12.123")
+ val v3 = Timestamp.valueOf("2017-08-21 23:42:12.0")
+
+ val data = Seq(v1, v2, v3)
+
+ val q1 = "select c1 from v where c1 > '2017-8-1'"
+ val q2 = "select c1 from v where c1 > '2017-08-21 23:42:12'"
+ val q3 = "select c1 from v where c1 > cast('2017-8-1' as timestamp)"
+
+ withTypeCoercionMode("hive") { spark =>
+ import spark.implicits._
+ data.toDF("c1").createTempView("v")
+ QueryTest.checkAnswer(spark.sql(q1), Row(v2) :: Row(v3) :: Nil)
+ QueryTest.checkAnswer(spark.sql(q2), Row(v2) :: Nil)
+ QueryTest.checkAnswer(spark.sql(q3), Row(v2) :: Row(v3) :: Nil)
+ }
+
+ withTypeCoercionMode("legacy") { spark =>
+ import spark.implicits._
+ data.toDF("c1").createTempView("v")
+ QueryTest.checkAnswer(spark.sql(q1), Nil)
+ QueryTest.checkAnswer(spark.sql(q2), Row(v2) :: Row(v3) :: Nil)
+ QueryTest.checkAnswer(spark.sql(q3), Row(v2) :: Row(v3) :: Nil)
+ }
+ }
+
+ test("CommonTypeForBinaryComparison: TimestampType vs DateType") {
+ val v1 = Timestamp.valueOf("2017-07-21 23:42:12.123")
+ val v2 = Timestamp.valueOf("2017-08-21 23:42:12.123")
+
+ val data = Seq(v1, v2)
+
+ val q1 = "select c1 from v where c1 > cast('2017-8-1' as date)"
+ val q2 = "select c1 from v where c1 > cast('2017-8-1' as timestamp)"
+
+ withTypeCoercionMode("hive") { spark =>
+ import spark.implicits._
+ data.toDF("c1").createTempView("v")
+ QueryTest.checkAnswer(spark.sql(q1), Row(v2) :: Nil)
+ QueryTest.checkAnswer(spark.sql(q2), Row(v2) :: Nil)
+ }
+
+ withTypeCoercionMode("legacy") { spark =>
+ import spark.implicits._
+ data.toDF("c1").createTempView("v")
+ QueryTest.checkAnswer(spark.sql(q1), Row(v2) :: Nil)
+ QueryTest.checkAnswer(spark.sql(q2), Row(v2) :: Nil)
+ }
+ }
+
+ test("CommonTypeForBinaryComparison: TimestampType vs NumericType") {
+ val v1 = Timestamp.valueOf("2017-07-21 23:42:12.123")
+ val v2 = Timestamp.valueOf("2017-08-21 23:42:12.123")
+
+ val data = Seq(v1, v2)
+
+ val q1 = "select c1 from v where c1 > 1"
+ val q2 = "select c1 from v where c1 > '2017-8-1'"
+ val q3 = "select c1 from v where c1 > '2017-08-01'"
+ val q4 = "select * from v where c1 > cast(cast('2017-08-01' as timestamp) as double)"
+
+ withTypeCoercionMode("hive") { spark =>
+ import spark.implicits._
+ data.toDF("c1").createTempView("v")
+ QueryTest.checkAnswer(spark.sql(q1), Row(v1) :: Row(v2) :: Nil)
+ QueryTest.checkAnswer(spark.sql(q2), Row(v2) :: Nil)
+ QueryTest.checkAnswer(spark.sql(q3), Row(v2) :: Nil)
+ QueryTest.checkAnswer(spark.sql(q4), Row(v2) :: Nil)
+ }
+
+ withTypeCoercionMode("legacy") { spark =>
+ import spark.implicits._
+ data.toDF("c1").createTempView("v")
+ val e1 = intercept[AnalysisException] {
+ spark.sql(q1)
+ }
+ assert(e1.getMessage.contains("data type mismatch"))
+ QueryTest.checkAnswer(spark.sql(q2), Nil)
+ QueryTest.checkAnswer(spark.sql(q3), Row(v2) :: Nil)
+ val e2 = intercept[AnalysisException] {
+ spark.sql(q4)
+ }
+ assert(e2.getMessage.contains("data type mismatch"))
+ }
+ }
+}
diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/TypeCoercionSuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/TypeCoercionSuite.scala
deleted file mode 100644
index 748baf31b905f..0000000000000
--- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/TypeCoercionSuite.scala
+++ /dev/null
@@ -1,171 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.hive.execution
-
-import java.sql.{Date, Timestamp}
-
-import org.apache.spark.SparkConf
-import org.apache.spark.sql._
-import org.apache.spark.sql.LocalSparkSession.withSparkSession
-import org.apache.spark.sql.internal.SQLConf
-
-class TypeCoercionSuite extends QueryTest {
-
- val spark: SparkSession = null
-
- val hiveSparkConf = new SparkConf()
- .setMaster("local")
- .set(SQLConf.typeCoercionMode.key, "hive")
-
- val legacySparkConf = new SparkConf()
- .setMaster("local")
- .set(SQLConf.typeCoercionMode.key, "legacy")
-
- test("SPARK-21646: CommonTypeForBinaryComparison: StringType vs NumericType") {
- val str1 = Long.MaxValue.toString + "1"
- val str2 = Int.MaxValue.toString + "1"
- val str3 = "10"
- val str4 = "0"
- val str5 = "-0.4"
- val str6 = "0.6"
-
- withSparkSession(SparkSession.builder.config(hiveSparkConf).getOrCreate()) { spark =>
- import spark.implicits._
- Seq(str1, str2, str3, str4, str5, str6).toDF("c1").createOrReplaceTempView("v")
- checkAnswer(spark.sql("SELECT c1 from v where c1 > 0"),
- Row(str1) :: Row(str2) :: Row(str3) :: Row(str6) :: Nil)
- checkAnswer(spark.sql("SELECT c1 from v where c1 > 0L"),
- Row(str1) :: Row(str2) :: Row(str3) :: Row(str6) :: Nil)
- checkAnswer(spark.sql("SELECT c1 FROM v WHERE c1 = 0"),
- Seq(Row("0")))
- }
-
- withSparkSession(SparkSession.builder.config(legacySparkConf).getOrCreate()) { spark =>
- import spark.implicits._
- Seq(str1, str2, str3, str4, str5, str6).toDF("c1").createOrReplaceTempView("v")
- checkAnswer(spark.sql("SELECT c1 from v where c1 > 0"),
- Row(str3) :: Nil)
- checkAnswer(spark.sql("SELECT c1 from v where c1 > 0L"),
- Row(str2) :: Row(str3) :: Nil)
- checkAnswer(spark.sql("SELECT c1 FROM v WHERE c1 = 0"),
- Seq(Row("0"), Row("-0.4"), Row("0.6")))
- }
- }
-
- test("SPARK-21646: CommonTypeForBinaryComparison: StringType vs DateType") {
- val v1 = Date.valueOf("2017-09-22")
- val v2 = Date.valueOf("2017-09-09")
-
- withSparkSession(SparkSession.builder.config(hiveSparkConf).getOrCreate()) { spark =>
- import spark.implicits._
- Seq(v1, v2).toDF("c1").createTempView("v")
- checkAnswer(spark.sql("select c1 from v where c1 > '2017-8-1'"), Row(v1) :: Row(v2) :: Nil)
- checkAnswer(spark.sql("select c1 from v where c1 > '2014'"), Row(v1) :: Row(v2) :: Nil)
- checkAnswer(spark.sql("select c1 from v where c1 > cast('2017-8-1' as date)"),
- Row(v1) :: Row(v2) :: Nil)
- }
-
- withSparkSession(SparkSession.builder.config(legacySparkConf).getOrCreate()) { spark =>
- import spark.implicits._
- Seq(v1, v2).toDF("c1").createTempView("v")
- checkAnswer(spark.sql("select c1 from v where c1 > '2017-8-1'"), Nil)
- checkAnswer(spark.sql("select c1 from v where c1 > '2014'"), Row(v1) :: Row(v2) :: Nil)
- checkAnswer(spark.sql("select c1 from v where c1 > cast('2017-8-1' as date)"),
- Row(v1) :: Row(v2) :: Nil)
- }
- }
-
- test("SPARK-21646: CommonTypeForBinaryComparison: StringType vs TimestampType") {
- val v1 = Timestamp.valueOf("2017-07-21 23:42:12.123")
- val v2 = Timestamp.valueOf("2017-08-21 23:42:12.123")
- val v3 = Timestamp.valueOf("2017-08-21 23:42:12.0")
-
- withSparkSession(SparkSession.builder.config(hiveSparkConf).getOrCreate()) { spark =>
- import spark.implicits._
- Seq(v1, v2, v3).toDF("c1").createTempView("v")
- checkAnswer(spark.sql("select c1 from v where c1 > '2017-8-1'"), Row(v2) :: Row(v3) :: Nil)
- checkAnswer(spark.sql("select c1 from v where c1 > '2017-08-21 23:42:12'"),
- Row(v2) :: Nil)
- checkAnswer(spark.sql("select c1 from v where c1 > cast('2017-8-1' as timestamp)"),
- Row(v2) :: Row(v3) :: Nil)
- }
-
- withSparkSession(SparkSession.builder.config(legacySparkConf).getOrCreate()) { spark =>
- import spark.implicits._
- Seq(v1, v2, v3).toDF("c1").createTempView("v")
- checkAnswer(spark.sql("select c1 from v where c1 > '2017-8-1'"), Nil)
- checkAnswer(spark.sql("select c1 from v where c1 >= '2017-08-21 23:42:12'"),
- Row(v2) :: Row(v3) :: Nil)
- checkAnswer(spark.sql("select c1 from v where c1 > cast('2017-8-1' as timestamp)"),
- Row(v2) :: Row(v3) :: Nil)
- }
- }
-
- test("SPARK-21646: CommonTypeForBinaryComparison: TimestampType vs DateType") {
- val v1 = Timestamp.valueOf("2017-07-21 23:42:12.123")
- val v2 = Timestamp.valueOf("2017-08-21 23:42:12.123")
-
- withSparkSession(SparkSession.builder.config(hiveSparkConf).getOrCreate()) { spark =>
- import spark.implicits._
- Seq(v1, v2).toDF("c1").createTempView("v")
- checkAnswer(spark.sql("select c1 from v where c1 > cast('2017-8-1' as date)"), Row(v2) :: Nil)
- checkAnswer(spark.sql("select c1 from v where c1 > cast('2017-8-1' as timestamp)"),
- Row(v2) :: Nil)
- }
-
- withSparkSession(SparkSession.builder.config(legacySparkConf).getOrCreate()) { spark =>
- import spark.implicits._
- Seq(v1, v2).toDF("c1").createTempView("v")
- checkAnswer(spark.sql("select c1 from v where c1 > cast('2017-8-1' as date)"), Row(v2) :: Nil)
- checkAnswer(spark.sql("select c1 from v where c1 > cast('2017-8-1' as timestamp)"),
- Row(v2) :: Nil)
- }
- }
-
- test("SPARK-21646: CommonTypeForBinaryComparison: TimestampType vs NumericType") {
- val v1 = Timestamp.valueOf("2017-07-21 23:42:12.123")
- val v2 = Timestamp.valueOf("2017-08-21 23:42:12.123")
-
- withSparkSession(SparkSession.builder.config(hiveSparkConf).getOrCreate()) { spark =>
- import spark.implicits._
- Seq(v1, v2).toDF("c1").createTempView("v")
- checkAnswer(spark.sql("select c1 from v where c1 > 1"), Row(v1) :: Row(v2) :: Nil)
- checkAnswer(spark.sql("select c1 from v where c1 > '2017-8-1'"), Row(v2) :: Nil)
- checkAnswer(spark.sql("select c1 from v where c1 > '2017-08-01'"), Row(v2) :: Nil)
- checkAnswer(
- spark.sql("select * from v where c1 > cast(cast('2017-08-01' as timestamp) as double)"),
- Row(v2) :: Nil)
- }
-
- withSparkSession(SparkSession.builder.config(legacySparkConf).getOrCreate()) { spark =>
- import spark.implicits._
- Seq(v1, v2).toDF("c1").createTempView("v")
- val e1 = intercept[AnalysisException] {
- spark.sql("select * from v where c1 > 1")
- }
- assert(e1.getMessage.contains("data type mismatch"))
- checkAnswer(spark.sql("select c1 from v where c1 > '2017-8-1'"), Nil)
- checkAnswer(spark.sql("select c1 from v where c1 > '2017-08-01'"), Row(v2) :: Nil)
- val e2 = intercept[AnalysisException] {
- spark.sql("select * from v where c1 > cast(cast('2017-08-01' as timestamp) as double)")
- }
- assert(e2.getMessage.contains("data type mismatch"))
- }
- }
-
-}
From d34f294affe2722811e9e09ff2e4d289c35b1319 Mon Sep 17 00:00:00 2001
From: Yuming Wang
Date: Thu, 12 Oct 2017 19:22:16 +0800
Subject: [PATCH 12/19] Add IN test suite
---
.../sql/execution/TypeCoercionModeSuite.scala | 100 ++++++++++--------
1 file changed, 55 insertions(+), 45 deletions(-)
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/TypeCoercionModeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/TypeCoercionModeSuite.scala
index b71813111121c..09afed71a92ec 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/TypeCoercionModeSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/TypeCoercionModeSuite.scala
@@ -43,9 +43,16 @@ class TypeCoercionModeSuite extends SparkFunSuite with BeforeAndAfterAll {
originalInstantiatedSparkSession.foreach(ctx => SparkSession.setDefaultSession(ctx))
}
- var sparkSession: SparkSession = _
+ private def checkAnswer(actual: => DataFrame, expectedAnswer: Seq[Row]): Unit = {
+ QueryTest.checkAnswer(actual, expectedAnswer) match {
+ case Some(errorMessage) => fail(errorMessage)
+ case None =>
+ }
+ }
+
+ private var sparkSession: SparkSession = _
- def withTypeCoercionMode[T](typeCoercionMode: String)(f: SparkSession => T): T = {
+ private def withTypeCoercionMode[T](typeCoercionMode: String)(f: SparkSession => T): T = {
try {
val sparkConf = new SparkConf(false)
.setMaster("local")
@@ -74,24 +81,27 @@ class TypeCoercionModeSuite extends SparkFunSuite with BeforeAndAfterAll {
val data = Seq(str1, str2, str3, str4, str5, str6)
- val q1 = "SELECT c1 from v where c1 > 0"
- val q2 = "SELECT c1 from v where c1 > 0L"
+ val q1 = "SELECT c1 FROM v WHERE c1 > 0"
+ val q2 = "SELECT c1 FROM v WHERE c1 > 0L"
val q3 = "SELECT c1 FROM v WHERE c1 = 0"
+ val q4 = "SELECT c1 FROM v WHERE c1 in (0)"
withTypeCoercionMode("hive") { spark =>
import spark.implicits._
data.toDF("c1").createOrReplaceTempView("v")
- QueryTest.checkAnswer(spark.sql(q1), Row(str1) :: Row(str2) :: Row(str3) :: Row(str6) :: Nil)
- QueryTest.checkAnswer(spark.sql(q2), Row(str1) :: Row(str2) :: Row(str3) :: Row(str6) :: Nil)
- QueryTest.checkAnswer(spark.sql(q3), Seq(Row(str4)))
+ checkAnswer(spark.sql(q1), Row(str1) :: Row(str2) :: Row(str3) :: Row(str6) :: Nil)
+ checkAnswer(spark.sql(q2), Row(str1) :: Row(str2) :: Row(str3) :: Row(str6) :: Nil)
+ checkAnswer(spark.sql(q3), Row(str4) :: Nil)
+ checkAnswer(spark.sql(q4), Row(str4) :: Nil)
}
withTypeCoercionMode("legacy") { spark =>
import spark.implicits._
data.toDF("c1").createOrReplaceTempView("v")
- QueryTest.checkAnswer(spark.sql(q1), Row(str3) :: Nil)
- QueryTest.checkAnswer(spark.sql(q2), Row(str2) :: Row(str3) :: Nil)
- QueryTest.checkAnswer(spark.sql(q3), Row(str4) :: Row(str5) :: Row(str6) :: Nil)
+ checkAnswer(spark.sql(q1), Row(str3) :: Nil)
+ checkAnswer(spark.sql(q2), Row(str2) :: Row(str3) :: Nil)
+ checkAnswer(spark.sql(q3), Row(str4) :: Row(str5) :: Row(str6) :: Nil)
+ checkAnswer(spark.sql(q4), Row(str4) :: Nil)
}
}
@@ -101,52 +111,52 @@ class TypeCoercionModeSuite extends SparkFunSuite with BeforeAndAfterAll {
val data = Seq(v1, v2)
- val q1 = "select c1 from v where c1 > '2017-8-1'"
- val q2 = "select c1 from v where c1 > '2014'"
- val q3 = "select c1 from v where c1 > cast('2017-8-1' as date)"
+ val q1 = "SELECT c1 FROM v WHERE c1 > '2017-8-1'"
+ val q2 = "SELECT c1 FROM v WHERE c1 > '2014'"
+ val q3 = "SELECT c1 FROM v WHERE c1 > cast('2017-8-1' as date)"
withTypeCoercionMode("hive") { spark =>
import spark.implicits._
data.toDF("c1").createTempView("v")
- QueryTest.checkAnswer(spark.sql(q1), Row(v1) :: Row(v2) :: Nil)
- QueryTest.checkAnswer(spark.sql(q2), Row(v1) :: Row(v2) :: Nil)
- QueryTest.checkAnswer(spark.sql(q3), Row(v1) :: Row(v2) :: Nil)
+ checkAnswer(spark.sql(q1), Row(v1) :: Row(v2) :: Nil)
+ checkAnswer(spark.sql(q2), Row(v1) :: Row(v2) :: Nil)
+ checkAnswer(spark.sql(q3), Row(v1) :: Row(v2) :: Nil)
}
withTypeCoercionMode("legacy") { spark =>
import spark.implicits._
data.toDF("c1").createTempView("v")
- QueryTest.checkAnswer(spark.sql(q1), Nil)
- QueryTest.checkAnswer(spark.sql(q2), Row(v1) :: Row(v2) :: Nil)
- QueryTest.checkAnswer(spark.sql(q3), Row(v1) :: Row(v2) :: Nil)
+ checkAnswer(spark.sql(q1), Nil)
+ checkAnswer(spark.sql(q2), Row(v1) :: Row(v2) :: Nil)
+ checkAnswer(spark.sql(q3), Row(v1) :: Row(v2) :: Nil)
}
}
test("CommonTypeForBinaryComparison: StringType vs TimestampType") {
val v1 = Timestamp.valueOf("2017-07-21 23:42:12.123")
val v2 = Timestamp.valueOf("2017-08-21 23:42:12.123")
- val v3 = Timestamp.valueOf("2017-08-21 23:42:12.0")
+ val v3 = Timestamp.valueOf("2017-08-21 23:42:12")
val data = Seq(v1, v2, v3)
- val q1 = "select c1 from v where c1 > '2017-8-1'"
- val q2 = "select c1 from v where c1 > '2017-08-21 23:42:12'"
- val q3 = "select c1 from v where c1 > cast('2017-8-1' as timestamp)"
+ val q1 = "SELECT c1 FROM v WHERE c1 > '2017-8-1'"
+ val q2 = "SELECT c1 FROM v WHERE c1 < '2017-08-21 23:42:12.0'"
+ val q3 = "SELECT c1 FROM v WHERE c1 > cast('2017-8-1' as timestamp)"
withTypeCoercionMode("hive") { spark =>
import spark.implicits._
data.toDF("c1").createTempView("v")
- QueryTest.checkAnswer(spark.sql(q1), Row(v2) :: Row(v3) :: Nil)
- QueryTest.checkAnswer(spark.sql(q2), Row(v2) :: Nil)
- QueryTest.checkAnswer(spark.sql(q3), Row(v2) :: Row(v3) :: Nil)
+ checkAnswer(spark.sql(q1), Row(v2) :: Row(v3) :: Nil)
+ checkAnswer(spark.sql(q2), Row(v1) :: Nil)
+ checkAnswer(spark.sql(q3), Row(v2) :: Row(v3) :: Nil)
}
withTypeCoercionMode("legacy") { spark =>
import spark.implicits._
data.toDF("c1").createTempView("v")
- QueryTest.checkAnswer(spark.sql(q1), Nil)
- QueryTest.checkAnswer(spark.sql(q2), Row(v2) :: Row(v3) :: Nil)
- QueryTest.checkAnswer(spark.sql(q3), Row(v2) :: Row(v3) :: Nil)
+ checkAnswer(spark.sql(q1), Nil)
+ checkAnswer(spark.sql(q2), Row(v1) :: Row(v3) :: Nil)
+ checkAnswer(spark.sql(q3), Row(v2) :: Row(v3) :: Nil)
}
}
@@ -156,21 +166,21 @@ class TypeCoercionModeSuite extends SparkFunSuite with BeforeAndAfterAll {
val data = Seq(v1, v2)
- val q1 = "select c1 from v where c1 > cast('2017-8-1' as date)"
- val q2 = "select c1 from v where c1 > cast('2017-8-1' as timestamp)"
+ val q1 = "SELECT c1 FROM v WHERE c1 > cast('2017-8-1' as date)"
+ val q2 = "SELECT c1 FROM v WHERE c1 > cast('2017-8-1' as timestamp)"
withTypeCoercionMode("hive") { spark =>
import spark.implicits._
data.toDF("c1").createTempView("v")
- QueryTest.checkAnswer(spark.sql(q1), Row(v2) :: Nil)
- QueryTest.checkAnswer(spark.sql(q2), Row(v2) :: Nil)
+ checkAnswer(spark.sql(q1), Row(v2) :: Nil)
+ checkAnswer(spark.sql(q2), Row(v2) :: Nil)
}
withTypeCoercionMode("legacy") { spark =>
import spark.implicits._
data.toDF("c1").createTempView("v")
- QueryTest.checkAnswer(spark.sql(q1), Row(v2) :: Nil)
- QueryTest.checkAnswer(spark.sql(q2), Row(v2) :: Nil)
+ checkAnswer(spark.sql(q1), Row(v2) :: Nil)
+ checkAnswer(spark.sql(q2), Row(v2) :: Nil)
}
}
@@ -180,18 +190,18 @@ class TypeCoercionModeSuite extends SparkFunSuite with BeforeAndAfterAll {
val data = Seq(v1, v2)
- val q1 = "select c1 from v where c1 > 1"
- val q2 = "select c1 from v where c1 > '2017-8-1'"
- val q3 = "select c1 from v where c1 > '2017-08-01'"
- val q4 = "select * from v where c1 > cast(cast('2017-08-01' as timestamp) as double)"
+ val q1 = "SELECT c1 FROM v WHERE c1 > 1"
+ val q2 = "SELECT c1 FROM v WHERE c1 > '2017-8-1'"
+ val q3 = "SELECT c1 FROM v WHERE c1 > '2017-08-01'"
+ val q4 = "SELECT c1 FROM v WHERE c1 > cast(cast('2017-08-01' as timestamp) as double)"
withTypeCoercionMode("hive") { spark =>
import spark.implicits._
data.toDF("c1").createTempView("v")
- QueryTest.checkAnswer(spark.sql(q1), Row(v1) :: Row(v2) :: Nil)
- QueryTest.checkAnswer(spark.sql(q2), Row(v2) :: Nil)
- QueryTest.checkAnswer(spark.sql(q3), Row(v2) :: Nil)
- QueryTest.checkAnswer(spark.sql(q4), Row(v2) :: Nil)
+ checkAnswer(spark.sql(q1), Row(v1) :: Row(v2) :: Nil)
+ checkAnswer(spark.sql(q2), Row(v2) :: Nil)
+ checkAnswer(spark.sql(q3), Row(v2) :: Nil)
+ checkAnswer(spark.sql(q4), Row(v2) :: Nil)
}
withTypeCoercionMode("legacy") { spark =>
@@ -201,8 +211,8 @@ class TypeCoercionModeSuite extends SparkFunSuite with BeforeAndAfterAll {
spark.sql(q1)
}
assert(e1.getMessage.contains("data type mismatch"))
- QueryTest.checkAnswer(spark.sql(q2), Nil)
- QueryTest.checkAnswer(spark.sql(q3), Row(v2) :: Nil)
+ checkAnswer(spark.sql(q2), Nil)
+ checkAnswer(spark.sql(q3), Row(v2) :: Nil)
val e2 = intercept[AnalysisException] {
spark.sql(q4)
}
From 0d9cf697c8b103273642bd7a8e2671dda0a04c5d Mon Sep 17 00:00:00 2001
From: Yuming Wang
Date: Tue, 14 Nov 2017 13:07:36 +0800
Subject: [PATCH 13/19] legacy -> default
---
docs/sql-programming-guide.md | 4 ++--
.../scala/org/apache/spark/sql/internal/SQLConf.scala | 6 +++---
.../spark/sql/execution/TypeCoercionModeSuite.scala | 10 +++++-----
3 files changed, 10 insertions(+), 10 deletions(-)
diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md
index 9cc304066b6fc..62bc653767282 100644
--- a/docs/sql-programming-guide.md
+++ b/docs/sql-programming-guide.md
@@ -1492,9 +1492,9 @@ that these options will be deprecated in future release as more optimizations ar
spark.sql.typeCoercion.mode |
- legacy |
+ default |
- The legacy type coercion mode was used in spark prior to 2.3, and so it continues to be the default to avoid breaking behavior. However, it has logical inconsistencies. The hive mode is preferred for most new applications, though it may require additional manual casting.
+ The default type coercion mode was used in spark prior to 2.3, and so it continues to be the default to avoid breaking behavior. However, it has logical inconsistencies. The hive mode is preferred for most new applications, though it may require additional manual casting.
|
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
index 17bfa9eb252fc..4233666dcf9eb 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
@@ -950,15 +950,15 @@ object SQLConf {
val typeCoercionMode =
buildConf("spark.sql.typeCoercion.mode")
- .doc("The 'legacy' typeCoercion mode was used in spark prior to 2.3, " +
+ .doc("The 'default' typeCoercion mode was used in spark prior to 2.3, " +
"and so it continues to be the default to avoid breaking behavior. " +
"However, it has logical inconsistencies. " +
"The 'hive' mode is preferred for most new applications, " +
"though it may require additional manual casting.")
.stringConf
.transform(_.toLowerCase(Locale.ROOT))
- .checkValues(Set("legacy", "hive"))
- .createWithDefault("legacy")
+ .checkValues(Set("default", "hive"))
+ .createWithDefault("default")
val REPLACE_EXCEPT_WITH_FILTER = buildConf("spark.sql.optimizer.replaceExceptWithFilter")
.internal()
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/TypeCoercionModeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/TypeCoercionModeSuite.scala
index 09afed71a92ec..eff87ee1f8038 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/TypeCoercionModeSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/TypeCoercionModeSuite.scala
@@ -95,7 +95,7 @@ class TypeCoercionModeSuite extends SparkFunSuite with BeforeAndAfterAll {
checkAnswer(spark.sql(q4), Row(str4) :: Nil)
}
- withTypeCoercionMode("legacy") { spark =>
+ withTypeCoercionMode("default") { spark =>
import spark.implicits._
data.toDF("c1").createOrReplaceTempView("v")
checkAnswer(spark.sql(q1), Row(str3) :: Nil)
@@ -123,7 +123,7 @@ class TypeCoercionModeSuite extends SparkFunSuite with BeforeAndAfterAll {
checkAnswer(spark.sql(q3), Row(v1) :: Row(v2) :: Nil)
}
- withTypeCoercionMode("legacy") { spark =>
+ withTypeCoercionMode("default") { spark =>
import spark.implicits._
data.toDF("c1").createTempView("v")
checkAnswer(spark.sql(q1), Nil)
@@ -151,7 +151,7 @@ class TypeCoercionModeSuite extends SparkFunSuite with BeforeAndAfterAll {
checkAnswer(spark.sql(q3), Row(v2) :: Row(v3) :: Nil)
}
- withTypeCoercionMode("legacy") { spark =>
+ withTypeCoercionMode("default") { spark =>
import spark.implicits._
data.toDF("c1").createTempView("v")
checkAnswer(spark.sql(q1), Nil)
@@ -176,7 +176,7 @@ class TypeCoercionModeSuite extends SparkFunSuite with BeforeAndAfterAll {
checkAnswer(spark.sql(q2), Row(v2) :: Nil)
}
- withTypeCoercionMode("legacy") { spark =>
+ withTypeCoercionMode("default") { spark =>
import spark.implicits._
data.toDF("c1").createTempView("v")
checkAnswer(spark.sql(q1), Row(v2) :: Nil)
@@ -204,7 +204,7 @@ class TypeCoercionModeSuite extends SparkFunSuite with BeforeAndAfterAll {
checkAnswer(spark.sql(q4), Row(v2) :: Nil)
}
- withTypeCoercionMode("legacy") { spark =>
+ withTypeCoercionMode("default") { spark =>
import spark.implicits._
data.toDF("c1").createTempView("v")
val e1 = intercept[AnalysisException] {
From 663eb3559c32058a2b85dc96a5fbf86ee5a6df4d Mon Sep 17 00:00:00 2001
From: Yuming Wang
Date: Wed, 6 Dec 2017 08:33:38 +0800
Subject: [PATCH 14/19] Update doc
---
docs/sql-programming-guide.md | 5 ++++-
.../spark/sql/catalyst/analysis/TypeCoercion.scala | 11 +----------
2 files changed, 5 insertions(+), 11 deletions(-)
diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md
index b8689262de018..f9f50f85ef818 100644
--- a/docs/sql-programming-guide.md
+++ b/docs/sql-programming-guide.md
@@ -1494,7 +1494,10 @@ that these options will be deprecated in future release as more optimizations ar
spark.sql.typeCoercion.mode |
default |
- The default type coercion mode was used in spark prior to 2.3, and so it continues to be the default to avoid breaking behavior. However, it has logical inconsistencies. The hive mode is preferred for most new applications, though it may require additional manual casting.
+ The default type coercion mode was used in spark prior to 2.3.0, and so it
+ continues to be the default to avoid breaking behavior. However, it has logical
+ inconsistencies. The hive mode is preferred for most new applications, though
+ it may require additional manual casting.
|
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 ee56065db54bf..9daf316bd4021 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
@@ -434,15 +434,6 @@ object TypeCoercion {
}
}
- private def flattenExpr(expr: Expression): Seq[Expression] = {
- expr match {
- // Multi columns in IN clause is represented as a CreateNamedStruct.
- // flatten the named struct to get the list of expressions.
- case cns: CreateNamedStruct => cns.valExprs
- case expr => Seq(expr)
- }
- }
-
/**
* Handles type coercion for both IN expression with subquery and IN
* expressions without subquery.
@@ -516,7 +507,7 @@ object TypeCoercion {
*/
object HiveInConversion extends TypeCoercionRule {
- def apply(plan: LogicalPlan): LogicalPlan = plan resolveExpressions {
+ override protected def coerceTypes(plan: LogicalPlan): LogicalPlan = plan resolveExpressions {
// Skip nodes who's children have not been resolved yet.
case e if !e.childrenResolved => e
From 7802483075e4328b3a2299413376cf2c6440a2b0 Mon Sep 17 00:00:00 2001
From: Yuming Wang
Date: Wed, 6 Dec 2017 10:18:42 +0800
Subject: [PATCH 15/19] Remove duplicate InConversion
---
.../org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala | 3 +--
1 file changed, 1 insertion(+), 2 deletions(-)
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 9daf316bd4021..6ed85d30c418f 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
@@ -47,8 +47,7 @@ import org.apache.spark.sql.types._
object TypeCoercion {
private val commonTypeCoercionRules =
- InConversion ::
- WidenSetOperationTypes ::
+ WidenSetOperationTypes ::
DecimalPrecision ::
BooleanEquality ::
FunctionArgumentConversion ::
From 97a071d91ec25159bba655b2bd9f6e2134d87088 Mon Sep 17 00:00:00 2001
From: Yuming Wang
Date: Tue, 9 Jan 2018 14:25:01 +0800
Subject: [PATCH 16/19] Merge SPARK-22894 to Hive mode.
---
.../apache/spark/sql/catalyst/analysis/TypeCoercion.scala | 6 ++++--
1 file changed, 4 insertions(+), 2 deletions(-)
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 01d2e3ec57725..bf67a9c21dca7 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
@@ -398,9 +398,11 @@ object TypeCoercion {
// Skip nodes who's children have not been resolved yet.
case e if !e.childrenResolved => e
- case a @ BinaryArithmetic(left @ StringType(), right) =>
+ case a @ BinaryArithmetic(left @ StringType(), right)
+ if right.dataType != CalendarIntervalType =>
a.makeCopy(Array(Cast(left, DoubleType), right))
- case a @ BinaryArithmetic(left, right @ StringType()) =>
+ case a @ BinaryArithmetic(left, right @ StringType())
+ if left.dataType != CalendarIntervalType =>
a.makeCopy(Array(left, Cast(right, DoubleType)))
case p @ Equality(left, right)
From 408e889caa8d61b7267f0f391be4af5fde82a0c9 Mon Sep 17 00:00:00 2001
From: Yuming Wang
Date: Tue, 9 Jan 2018 14:52:14 +0800
Subject: [PATCH 17/19] InConversion -> NativeInConversion; PromoteStrings ->
NativePromoteStrings
---
docs/sql-programming-guide.md | 6 ++----
.../spark/sql/catalyst/analysis/TypeCoercion.scala | 8 ++++----
.../scala/org/apache/spark/sql/internal/SQLConf.scala | 7 ++-----
.../sql/catalyst/analysis/TypeCoercionSuite.scala | 10 +++++-----
4 files changed, 13 insertions(+), 18 deletions(-)
diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md
index 5e2e29eb0506c..b1d6d3a868944 100644
--- a/docs/sql-programming-guide.md
+++ b/docs/sql-programming-guide.md
@@ -1497,10 +1497,8 @@ that these options will be deprecated in future release as more optimizations ar
spark.sql.typeCoercion.mode |
default |
- The default type coercion mode was used in spark prior to 2.3.0, and so it
- continues to be the default to avoid breaking behavior. However, it has logical
- inconsistencies. The hive mode is preferred for most new applications, though
- it may require additional manual casting.
+ Since Spark 2.3, the hive mode is introduced for Hive compatiblity.
+ Spark SQL has its native type cocersion mode, which is enabled by default.
|
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 bf67a9c21dca7..50aaba19feb27 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
@@ -67,8 +67,8 @@ object TypeCoercion {
HivePromoteStrings
} else {
commonTypeCoercionRules :+
- InConversion :+
- PromoteStrings
+ NativeInConversion :+
+ NativePromoteStrings
}
}
@@ -348,7 +348,7 @@ object TypeCoercion {
/**
* Promotes strings that appear in arithmetic expressions.
*/
- object PromoteStrings extends TypeCoercionRule {
+ object NativePromoteStrings extends TypeCoercionRule {
override protected def coerceTypes(
plan: LogicalPlan): LogicalPlan = plan transformAllExpressions {
@@ -451,7 +451,7 @@ object TypeCoercion {
* operator type is found the original expression will be returned and an
* Analysis Exception will be raised at the type checking phase.
*/
- object InConversion extends TypeCoercionRule {
+ object NativeInConversion extends TypeCoercionRule {
override protected def coerceTypes(
plan: LogicalPlan): LogicalPlan = plan transformAllExpressions {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
index 624311e1cbf91..c28eefb252aef 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
@@ -1032,11 +1032,8 @@ object SQLConf {
val typeCoercionMode =
buildConf("spark.sql.typeCoercion.mode")
- .doc("The 'default' typeCoercion mode was used in spark prior to 2.3, " +
- "and so it continues to be the default to avoid breaking behavior. " +
- "However, it has logical inconsistencies. " +
- "The 'hive' mode is preferred for most new applications, " +
- "though it may require additional manual casting.")
+ .doc("Since Spark 2.3, the 'hive' mode is introduced for Hive compatiblity. " +
+ "Spark SQL has its native type cocersion mode, which is enabled by default.")
.stringConf
.transform(_.toLowerCase(Locale.ROOT))
.checkValues(Set("default", "hive"))
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 52a7ebdafd7c7..f0261e0df2cf8 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
@@ -1207,7 +1207,7 @@ class TypeCoercionSuite extends AnalysisTest {
*/
test("make sure rules do not fire early") {
// InConversion
- val inConversion = TypeCoercion.InConversion
+ val inConversion = TypeCoercion.NativeInConversion
ruleTest(inConversion,
In(UnresolvedAttribute("a"), Seq(Literal(1))),
In(UnresolvedAttribute("a"), Seq(Literal(1)))
@@ -1251,16 +1251,16 @@ class TypeCoercionSuite extends AnalysisTest {
}
test("binary comparison with string promotion") {
- ruleTest(PromoteStrings,
+ ruleTest(NativePromoteStrings,
GreaterThan(Literal("123"), Literal(1)),
GreaterThan(Cast(Literal("123"), IntegerType), Literal(1)))
- ruleTest(PromoteStrings,
+ ruleTest(NativePromoteStrings,
LessThan(Literal(true), Literal("123")),
LessThan(Literal(true), Cast(Literal("123"), BooleanType)))
- ruleTest(PromoteStrings,
+ ruleTest(NativePromoteStrings,
EqualTo(Literal(Array(1, 2)), Literal("123")),
EqualTo(Literal(Array(1, 2)), Literal("123")))
- ruleTest(PromoteStrings,
+ ruleTest(NativePromoteStrings,
GreaterThan(Literal("1.5"), Literal(BigDecimal("0.5"))),
GreaterThan(Cast(Literal("1.5"), DoubleType), Cast(Literal(BigDecimal("0.5")), DoubleType)))
}
From e763330edae88d4dad410214608fb5448d90a989 Mon Sep 17 00:00:00 2001
From: Yuming Wang
Date: Tue, 9 Jan 2018 18:21:00 +0800
Subject: [PATCH 18/19] Lost WindowFrameCoercion
---
.../org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala | 3 ++-
1 file changed, 2 insertions(+), 1 deletion(-)
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 50aaba19feb27..afbb8965fb422 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
@@ -59,7 +59,8 @@ object TypeCoercion {
StackCoercion ::
Division ::
ImplicitTypeCasts ::
- DateTimeOperations :: Nil
+ DateTimeOperations ::
+ WindowFrameCoercion :: Nil
if (conf.isHiveTypeCoercionMode) {
commonTypeCoercionRules :+
From d0a2089b8c52a3be3977f0bfc5538758ef7d2b55 Mon Sep 17 00:00:00 2001
From: Yuming Wang
Date: Sun, 10 Jun 2018 20:18:53 +0800
Subject: [PATCH 19/19] Since Spark 2.3 -> Since Spark 2.4
---
docs/sql-programming-guide.md | 2 +-
.../sql/catalyst/analysis/TypeCoercion.scala | 17 ++++++++++-------
.../org/apache/spark/sql/internal/SQLConf.scala | 2 +-
3 files changed, 12 insertions(+), 9 deletions(-)
diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md
index 70d819c96ee99..92379539290d0 100644
--- a/docs/sql-programming-guide.md
+++ b/docs/sql-programming-guide.md
@@ -1528,7 +1528,7 @@ that these options will be deprecated in future release as more optimizations ar
spark.sql.typeCoercion.mode |
default |
- Since Spark 2.3, the hive mode is introduced for Hive compatiblity.
+ Since Spark 2.4, the hive mode is introduced for Hive compatiblity.
Spark SQL has its native type cocersion mode, which is enabled by default.
|
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 39d0285b985c6..9e27544427601 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
@@ -125,9 +125,9 @@ object TypeCoercion {
private def findCommonTypeForBinaryComparison(
dt1: DataType, dt2: DataType, conf: SQLConf): Option[DataType] = {
if (conf.isHiveTypeCoercionMode) {
- findCommonTypeToCompatibleWithHive(dt1, dt2)
+ findHiveCommonTypeForBinary(dt1, dt2)
} else {
- findCommonTypeForBinaryComparisonNative(dt1, dt2, conf)
+ findNativeCommonTypeForBinary(dt1, dt2, conf)
}
}
@@ -136,7 +136,7 @@ object TypeCoercion {
* is a String and the other is not. It also handles when one op is a Date and the
* other is a Timestamp by making the target type to be String.
*/
- private def findCommonTypeForBinaryComparisonNative(
+ private def findNativeCommonTypeForBinary(
dt1: DataType, dt2: DataType, conf: SQLConf): Option[DataType] = (dt1, dt2) match {
// We should cast all relative timestamp/date/string comparison into string comparisons
// This behaves as a user would expect because timestamp strings sort lexicographically.
@@ -167,10 +167,13 @@ object TypeCoercion {
case (l, r) => None
}
- val findCommonTypeToCompatibleWithHive: (DataType, DataType) => Option[DataType] = {
- // Follow hive's binary comparison action:
- // https://github.com/apache/hive/blob/rel/storage-release-2.4.0/ql/src/java/
- // org/apache/hadoop/hive/ql/exec/FunctionRegistry.java#L781
+ /**
+ * This function follow hive's binary comparison action:
+ * https://github.com/apache/hive/blob/rel/release-3.0.0/ql/src/java/
+ * org/apache/hadoop/hive/ql/exec/FunctionRegistry.java#L802
+ */
+ private def findHiveCommonTypeForBinary(
+ dt1: DataType, dt2: DataType): Option[DataType] = (dt1, dt2) match {
case (StringType, DateType) => Some(DateType)
case (DateType, StringType) => Some(DateType)
case (StringType, TimestampType) => Some(TimestampType)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
index 0f97be34b59f6..bfb06ccfbf85e 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
@@ -1118,7 +1118,7 @@ object SQLConf {
val typeCoercionMode =
buildConf("spark.sql.typeCoercion.mode")
- .doc("Since Spark 2.3, the 'hive' mode is introduced for Hive compatiblity. " +
+ .doc("Since Spark 2.4, the 'hive' mode is introduced for Hive compatiblity. " +
"Spark SQL has its native type cocersion mode, which is enabled by default.")
.stringConf
.transform(_.toLowerCase(Locale.ROOT))