Skip to content

Commit 6598161

Browse files
marmbrusyhuai
authored andcommitted
[SPARK-8420] [SQL] Fix comparision of timestamps/dates with strings (branch-1.4)
This is branch 1.4 backport of #6888. Below is the original description. In earlier versions of Spark SQL we casted `TimestampType` and `DataType` to `StringType` when it was involved in a binary comparison with a `StringType`. This allowed comparing a timestamp with a partial date as a user would expect. - `time > "2014-06-10"` - `time > "2014"` In 1.4.0 we tried to cast the String instead into a Timestamp. However, since partial dates are not a valid complete timestamp this results in `null` which results in the tuple being filtered. This PR restores the earlier behavior. Note that we still special case equality so that these comparisons are not affected by not printing zeros for subsecond precision. Author: Michael Armbrust <michaeldatabricks.com> Closes #6888 from marmbrus/timeCompareString and squashes the following commits: bdef29c [Michael Armbrust] test partial date 1f09adf [Michael Armbrust] special handling of equality 1172c60 [Michael Armbrust] more test fixing 4dfc412 [Michael Armbrust] fix tests aaa9508 [Michael Armbrust] newline 04d908f [Michael Armbrust] [SPARK-8420][SQL] Fix comparision of timestamps/dates with strings Conflicts: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala Author: Michael Armbrust <[email protected]> Closes #6914 from yhuai/timeCompareString-1.4 and squashes the following commits: 9882915 [Michael Armbrust] [SPARK-8420] [SQL] Fix comparision of timestamps/dates with strings
1 parent 451c872 commit 6598161

File tree

6 files changed

+88
-11
lines changed

6 files changed

+88
-11
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala

Lines changed: 14 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -242,7 +242,16 @@ trait HiveTypeCoercion {
242242
case a: BinaryArithmetic if a.right.dataType == StringType =>
243243
a.makeCopy(Array(a.left, Cast(a.right, DoubleType)))
244244

245-
// we should cast all timestamp/date/string compare into string compare
245+
// For equality between string and timestamp we cast the string to a timestamp
246+
// so that things like rounding of subsecond precision does not affect the comparison.
247+
case p @ Equality(left @ StringType(), right @ TimestampType()) =>
248+
p.makeCopy(Array(Cast(left, TimestampType), right))
249+
case p @ Equality(left @ TimestampType(), right @ StringType()) =>
250+
p.makeCopy(Array(left, Cast(right, TimestampType)))
251+
252+
// We should cast all relative timestamp/date/string comparison into string comparisions
253+
// This behaves as a user would expect because timestamp strings sort lexicographically.
254+
// i.e. TimeStamp(2013-01-01 00:00 ...) < "2014" = true
246255
case p: BinaryComparison if p.left.dataType == StringType &&
247256
p.right.dataType == DateType =>
248257
p.makeCopy(Array(p.left, Cast(p.right, StringType)))
@@ -251,10 +260,12 @@ trait HiveTypeCoercion {
251260
p.makeCopy(Array(Cast(p.left, StringType), p.right))
252261
case p: BinaryComparison if p.left.dataType == StringType &&
253262
p.right.dataType == TimestampType =>
254-
p.makeCopy(Array(Cast(p.left, TimestampType), p.right))
263+
p.makeCopy(Array(p.left, Cast(p.right, StringType)))
255264
case p: BinaryComparison if p.left.dataType == TimestampType &&
256265
p.right.dataType == StringType =>
257-
p.makeCopy(Array(p.left, Cast(p.right, TimestampType)))
266+
p.makeCopy(Array(Cast(p.left, StringType), p.right))
267+
268+
// Comparisons between dates and timestamps.
258269
case p: BinaryComparison if p.left.dataType == TimestampType &&
259270
p.right.dataType == DateType =>
260271
p.makeCopy(Array(Cast(p.left, StringType), Cast(p.right, StringType)))

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -175,6 +175,15 @@ abstract class BinaryComparison extends BinaryExpression with Predicate {
175175
self: Product =>
176176
}
177177

178+
/** An extractor that matches both standard 3VL equality and null-safe equality. */
179+
private[sql] object Equality {
180+
def unapply(e: BinaryComparison): Option[(Expression, Expression)] = e match {
181+
case EqualTo(l, r) => Some((l, r))
182+
case EqualNullSafe(l, r) => Some((l, r))
183+
case _ => None
184+
}
185+
}
186+
178187
case class EqualTo(left: Expression, right: Expression) extends BinaryComparison {
179188
override def symbol: String = "="
180189

Lines changed: 56 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,56 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.sql
19+
20+
import java.sql.{Date, Timestamp}
21+
22+
class DataFrameDateTimeSuite extends QueryTest {
23+
24+
private lazy val ctx = org.apache.spark.sql.test.TestSQLContext
25+
import ctx.implicits._
26+
27+
test("timestamp comparison with date strings") {
28+
val df = Seq(
29+
(1, Timestamp.valueOf("2015-01-01 00:00:00")),
30+
(2, Timestamp.valueOf("2014-01-01 00:00:00"))).toDF("i", "t")
31+
32+
checkAnswer(
33+
df.select("t").filter($"t" <= "2014-06-01"),
34+
Row(Timestamp.valueOf("2014-01-01 00:00:00")) :: Nil)
35+
36+
37+
checkAnswer(
38+
df.select("t").filter($"t" >= "2014-06-01"),
39+
Row(Timestamp.valueOf("2015-01-01 00:00:00")) :: Nil)
40+
}
41+
42+
test("date comparison with date strings") {
43+
val df = Seq(
44+
(1, Date.valueOf("2015-01-01")),
45+
(2, Date.valueOf("2014-01-01"))).toDF("i", "t")
46+
47+
checkAnswer(
48+
df.select("t").filter($"t" <= "2014-06-01"),
49+
Row(Date.valueOf("2014-01-01")) :: Nil)
50+
51+
52+
checkAnswer(
53+
df.select("t").filter($"t" >= "2015"),
54+
Row(Date.valueOf("2015-01-01")) :: Nil)
55+
}
56+
}

sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,8 @@ package org.apache.spark.sql
1919

2020
import org.scalatest.BeforeAndAfterAll
2121

22+
import java.sql.Timestamp
23+
2224
import org.apache.spark.sql.catalyst.DefaultParserDialect
2325
import org.apache.spark.sql.catalyst.errors.DialectException
2426
import org.apache.spark.sql.execution.GeneratedAggregate
@@ -311,6 +313,8 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll {
311313
}
312314

313315
test("SPARK-3173 Timestamp support in the parser") {
316+
(0 to 3).map(i => Tuple1(new Timestamp(i))).toDF("time").registerTempTable("timestamps")
317+
314318
checkAnswer(sql(
315319
"SELECT time FROM timestamps WHERE time='1969-12-31 16:00:00.0'"),
316320
Row(java.sql.Timestamp.valueOf("1969-12-31 16:00:00")))

sql/core/src/test/scala/org/apache/spark/sql/TestData.scala

Lines changed: 0 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -174,12 +174,6 @@ object TestData {
174174
"3, C3, true, null" ::
175175
"4, D4, true, 2147483644" :: Nil)
176176

177-
case class TimestampField(time: Timestamp)
178-
val timestamps = TestSQLContext.sparkContext.parallelize((0 to 3).map { i =>
179-
TimestampField(new Timestamp(i))
180-
})
181-
timestamps.toDF().registerTempTable("timestamps")
182-
183177
case class IntField(i: Int)
184178
// An RDD with 4 elements and 8 partitions
185179
val withEmptyParts = TestSQLContext.sparkContext.parallelize((1 to 4).map(IntField), 8)

sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala

Lines changed: 5 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -90,15 +90,18 @@ class InMemoryColumnarQuerySuite extends QueryTest {
9090
}
9191

9292
test("SPARK-2729 regression: timestamp data type") {
93+
val timestamps = (0 to 3).map(i => Tuple1(new Timestamp(i))).toDF("time")
94+
timestamps.registerTempTable("timestamps")
95+
9396
checkAnswer(
9497
sql("SELECT time FROM timestamps"),
95-
timestamps.collect().toSeq.map(Row.fromTuple))
98+
timestamps.collect().toSeq)
9699

97100
cacheTable("timestamps")
98101

99102
checkAnswer(
100103
sql("SELECT time FROM timestamps"),
101-
timestamps.collect().toSeq.map(Row.fromTuple))
104+
timestamps.collect().toSeq)
102105
}
103106

104107
test("SPARK-3320 regression: batched column buffer building should work with empty partitions") {

0 commit comments

Comments
 (0)