Skip to content

Commit 9456688

Browse files
tejasapatilgatorsmile
authored andcommitted
[SPARK-17495][SQL] Support date, timestamp and interval types in Hive hash
## What changes were proposed in this pull request? - Timestamp hashing is done as per [TimestampWritable.hashCode()](https://github.com/apache/hive/blob/ff67cdda1c538dc65087878eeba3e165cf3230f4/serde/src/java/org/apache/hadoop/hive/serde2/io/TimestampWritable.java#L406) in Hive - Interval hashing is done as per [HiveIntervalDayTime.hashCode()](https://github.com/apache/hive/blob/ff67cdda1c538dc65087878eeba3e165cf3230f4/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveIntervalDayTime.java#L178). Note that there are inherent differences in how Hive and Spark store intervals under the hood which limits the ability to be in completely sync with hive's hashing function. I have explained this in the method doc. - Date type was already supported. This PR adds test for that. ## How was this patch tested? Added unit tests Author: Tejas Patil <tejasp@fb.com> Closes #17062 from tejasapatil/SPARK-17495_time_related_types.
1 parent 0a4d06a commit 9456688

File tree

2 files changed

+268
-11
lines changed

2 files changed

+268
-11
lines changed

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

Lines changed: 62 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -335,6 +335,8 @@ abstract class HashExpression[E] extends Expression {
335335
}
336336
}
337337

338+
protected def genHashTimestamp(t: String, result: String): String = genHashLong(t, result)
339+
338340
protected def genHashCalendarInterval(input: String, result: String): String = {
339341
val microsecondsHash = s"$hasherClassName.hashLong($input.microseconds, $result)"
340342
s"$result = $hasherClassName.hashInt($input.months, $microsecondsHash);"
@@ -400,7 +402,8 @@ abstract class HashExpression[E] extends Expression {
400402
case NullType => ""
401403
case BooleanType => genHashBoolean(input, result)
402404
case ByteType | ShortType | IntegerType | DateType => genHashInt(input, result)
403-
case LongType | TimestampType => genHashLong(input, result)
405+
case LongType => genHashLong(input, result)
406+
case TimestampType => genHashTimestamp(input, result)
404407
case FloatType => genHashFloat(input, result)
405408
case DoubleType => genHashDouble(input, result)
406409
case d: DecimalType => genHashDecimal(ctx, d, input, result)
@@ -433,6 +436,10 @@ abstract class InterpretedHashFunction {
433436

434437
protected def hashUnsafeBytes(base: AnyRef, offset: Long, length: Int, seed: Long): Long
435438

439+
/**
440+
* Computes hash of a given `value` of type `dataType`. The caller needs to check the validity
441+
* of input `value`.
442+
*/
436443
def hash(value: Any, dataType: DataType, seed: Long): Long = {
437444
value match {
438445
case null => seed
@@ -580,8 +587,6 @@ object XxHash64Function extends InterpretedHashFunction {
580587
*
581588
* We should use this hash function for both shuffle and bucket of Hive tables, so that
582589
* we can guarantee shuffle and bucketing have same data distribution
583-
*
584-
* TODO: Support date related types
585590
*/
586591
@ExpressionDescription(
587592
usage = "_FUNC_(expr1, expr2, ...) - Returns a hash value of the arguments.")
@@ -648,11 +653,16 @@ case class HiveHash(children: Seq[Expression]) extends HashExpression[Int] {
648653

649654
override protected def genHashCalendarInterval(input: String, result: String): String = {
650655
s"""
651-
$result = (31 * $hasherClassName.hashInt($input.months)) +
652-
$hasherClassName.hashLong($input.microseconds);"
656+
$result = (int)
657+
${HiveHashFunction.getClass.getName.stripSuffix("$")}.hashCalendarInterval($input);
653658
"""
654659
}
655660

661+
override protected def genHashTimestamp(input: String, result: String): String =
662+
s"""
663+
$result = (int) ${HiveHashFunction.getClass.getName.stripSuffix("$")}.hashTimestamp($input);
664+
"""
665+
656666
override protected def genHashString(input: String, result: String): String = {
657667
val baseObject = s"$input.getBaseObject()"
658668
val baseOffset = s"$input.getBaseOffset()"
@@ -781,6 +791,49 @@ object HiveHashFunction extends InterpretedHashFunction {
781791
result
782792
}
783793

794+
/**
795+
* Mimics TimestampWritable.hashCode() in Hive
796+
*/
797+
def hashTimestamp(timestamp: Long): Long = {
798+
val timestampInSeconds = timestamp / 1000000
799+
val nanoSecondsPortion = (timestamp % 1000000) * 1000
800+
801+
var result = timestampInSeconds
802+
result <<= 30 // the nanosecond part fits in 30 bits
803+
result |= nanoSecondsPortion
804+
((result >>> 32) ^ result).toInt
805+
}
806+
807+
/**
808+
* Hive allows input intervals to be defined using units below but the intervals
809+
* have to be from the same category:
810+
* - year, month (stored as HiveIntervalYearMonth)
811+
* - day, hour, minute, second, nanosecond (stored as HiveIntervalDayTime)
812+
*
813+
* eg. (INTERVAL '30' YEAR + INTERVAL '-23' DAY) fails in Hive
814+
*
815+
* This method mimics HiveIntervalDayTime.hashCode() in Hive.
816+
*
817+
* Two differences wrt Hive due to how intervals are stored in Spark vs Hive:
818+
*
819+
* - If the `INTERVAL` is backed as HiveIntervalYearMonth in Hive, then this method will not
820+
* produce Hive compatible result. The reason being Spark's representation of calendar does not
821+
* have such categories based on the interval and is unified.
822+
*
823+
* - Spark's [[CalendarInterval]] has precision upto microseconds but Hive's
824+
* HiveIntervalDayTime can store data with precision upto nanoseconds. So, any input intervals
825+
* with nanosecond values will lead to wrong output hashes (ie. non adherent with Hive output)
826+
*/
827+
def hashCalendarInterval(calendarInterval: CalendarInterval): Long = {
828+
val totalSeconds = calendarInterval.microseconds / CalendarInterval.MICROS_PER_SECOND.toInt
829+
val result: Int = (17 * 37) + (totalSeconds ^ totalSeconds >> 32).toInt
830+
831+
val nanoSeconds =
832+
(calendarInterval.microseconds -
833+
(totalSeconds * CalendarInterval.MICROS_PER_SECOND.toInt)).toInt * 1000
834+
(result * 37) + nanoSeconds
835+
}
836+
784837
override def hash(value: Any, dataType: DataType, seed: Long): Long = {
785838
value match {
786839
case null => 0
@@ -834,10 +887,10 @@ object HiveHashFunction extends InterpretedHashFunction {
834887
}
835888
result
836889

837-
case d: Decimal =>
838-
normalizeDecimal(d.toJavaBigDecimal).hashCode()
839-
840-
case _ => super.hash(value, dataType, seed)
890+
case d: Decimal => normalizeDecimal(d.toJavaBigDecimal).hashCode()
891+
case timestamp: Long if dataType.isInstanceOf[TimestampType] => hashTimestamp(timestamp)
892+
case calendarInterval: CalendarInterval => hashCalendarInterval(calendarInterval)
893+
case _ => super.hash(value, dataType, 0)
841894
}
842895
}
843896
}

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala

Lines changed: 206 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -18,18 +18,20 @@
1818
package org.apache.spark.sql.catalyst.expressions
1919

2020
import java.nio.charset.StandardCharsets
21+
import java.util.TimeZone
2122

2223
import scala.collection.mutable.ArrayBuffer
2324

2425
import org.apache.commons.codec.digest.DigestUtils
26+
import org.scalatest.exceptions.TestFailedException
2527

2628
import org.apache.spark.SparkFunSuite
2729
import org.apache.spark.sql.{RandomDataGenerator, Row}
2830
import org.apache.spark.sql.catalyst.encoders.{ExamplePointUDT, RowEncoder}
2931
import org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection
30-
import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData}
32+
import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, DateTimeUtils, GenericArrayData}
3133
import org.apache.spark.sql.types.{ArrayType, StructType, _}
32-
import org.apache.spark.unsafe.types.UTF8String
34+
import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
3335

3436
class HashExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
3537
val random = new scala.util.Random
@@ -168,6 +170,208 @@ class HashExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
168170
// scalastyle:on nonascii
169171
}
170172

173+
test("hive-hash for date type") {
174+
def checkHiveHashForDateType(dateString: String, expected: Long): Unit = {
175+
checkHiveHash(
176+
DateTimeUtils.stringToDate(UTF8String.fromString(dateString)).get,
177+
DateType,
178+
expected)
179+
}
180+
181+
// basic case
182+
checkHiveHashForDateType("2017-01-01", 17167)
183+
184+
// boundary cases
185+
checkHiveHashForDateType("0000-01-01", -719530)
186+
checkHiveHashForDateType("9999-12-31", 2932896)
187+
188+
// epoch
189+
checkHiveHashForDateType("1970-01-01", 0)
190+
191+
// before epoch
192+
checkHiveHashForDateType("1800-01-01", -62091)
193+
194+
// Invalid input: bad date string. Hive returns 0 for such cases
195+
intercept[NoSuchElementException](checkHiveHashForDateType("0-0-0", 0))
196+
intercept[NoSuchElementException](checkHiveHashForDateType("-1212-01-01", 0))
197+
intercept[NoSuchElementException](checkHiveHashForDateType("2016-99-99", 0))
198+
199+
// Invalid input: Empty string. Hive returns 0 for this case
200+
intercept[NoSuchElementException](checkHiveHashForDateType("", 0))
201+
202+
// Invalid input: February 30th for a leap year. Hive supports this but Spark doesn't
203+
intercept[NoSuchElementException](checkHiveHashForDateType("2016-02-30", 16861))
204+
}
205+
206+
test("hive-hash for timestamp type") {
207+
def checkHiveHashForTimestampType(
208+
timestamp: String,
209+
expected: Long,
210+
timeZone: TimeZone = TimeZone.getTimeZone("UTC")): Unit = {
211+
checkHiveHash(
212+
DateTimeUtils.stringToTimestamp(UTF8String.fromString(timestamp), timeZone).get,
213+
TimestampType,
214+
expected)
215+
}
216+
217+
// basic case
218+
checkHiveHashForTimestampType("2017-02-24 10:56:29", 1445725271)
219+
220+
// with higher precision
221+
checkHiveHashForTimestampType("2017-02-24 10:56:29.111111", 1353936655)
222+
223+
// with different timezone
224+
checkHiveHashForTimestampType("2017-02-24 10:56:29", 1445732471,
225+
TimeZone.getTimeZone("US/Pacific"))
226+
227+
// boundary cases
228+
checkHiveHashForTimestampType("0001-01-01 00:00:00", 1645926784)
229+
checkHiveHashForTimestampType("9999-01-01 00:00:00", -1081818240)
230+
231+
// epoch
232+
checkHiveHashForTimestampType("1970-01-01 00:00:00", 0)
233+
234+
// before epoch
235+
checkHiveHashForTimestampType("1800-01-01 03:12:45", -267420885)
236+
237+
// Invalid input: bad timestamp string. Hive returns 0 for such cases
238+
intercept[NoSuchElementException](checkHiveHashForTimestampType("0-0-0 0:0:0", 0))
239+
intercept[NoSuchElementException](checkHiveHashForTimestampType("-99-99-99 99:99:45", 0))
240+
intercept[NoSuchElementException](checkHiveHashForTimestampType("555555-55555-5555", 0))
241+
242+
// Invalid input: Empty string. Hive returns 0 for this case
243+
intercept[NoSuchElementException](checkHiveHashForTimestampType("", 0))
244+
245+
// Invalid input: February 30th is a leap year. Hive supports this but Spark doesn't
246+
intercept[NoSuchElementException](checkHiveHashForTimestampType("2016-02-30 00:00:00", 0))
247+
248+
// Invalid input: Hive accepts upto 9 decimal place precision but Spark uses upto 6
249+
intercept[TestFailedException](checkHiveHashForTimestampType("2017-02-24 10:56:29.11111111", 0))
250+
}
251+
252+
test("hive-hash for CalendarInterval type") {
253+
def checkHiveHashForIntervalType(interval: String, expected: Long): Unit = {
254+
checkHiveHash(CalendarInterval.fromString(interval), CalendarIntervalType, expected)
255+
}
256+
257+
// ----- MICROSEC -----
258+
259+
// basic case
260+
checkHiveHashForIntervalType("interval 1 microsecond", 24273)
261+
262+
// negative
263+
checkHiveHashForIntervalType("interval -1 microsecond", 22273)
264+
265+
// edge / boundary cases
266+
checkHiveHashForIntervalType("interval 0 microsecond", 23273)
267+
checkHiveHashForIntervalType("interval 999 microsecond", 1022273)
268+
checkHiveHashForIntervalType("interval -999 microsecond", -975727)
269+
270+
// ----- MILLISEC -----
271+
272+
// basic case
273+
checkHiveHashForIntervalType("interval 1 millisecond", 1023273)
274+
275+
// negative
276+
checkHiveHashForIntervalType("interval -1 millisecond", -976727)
277+
278+
// edge / boundary cases
279+
checkHiveHashForIntervalType("interval 0 millisecond", 23273)
280+
checkHiveHashForIntervalType("interval 999 millisecond", 999023273)
281+
checkHiveHashForIntervalType("interval -999 millisecond", -998976727)
282+
283+
// ----- SECOND -----
284+
285+
// basic case
286+
checkHiveHashForIntervalType("interval 1 second", 23310)
287+
288+
// negative
289+
checkHiveHashForIntervalType("interval -1 second", 23273)
290+
291+
// edge / boundary cases
292+
checkHiveHashForIntervalType("interval 0 second", 23273)
293+
checkHiveHashForIntervalType("interval 2147483647 second", -2147460412)
294+
checkHiveHashForIntervalType("interval -2147483648 second", -2147460412)
295+
296+
// Out of range for both Hive and Spark
297+
// Hive throws an exception. Spark overflows and returns wrong output
298+
// checkHiveHashForIntervalType("interval 9999999999 second", 0)
299+
300+
// ----- MINUTE -----
301+
302+
// basic cases
303+
checkHiveHashForIntervalType("interval 1 minute", 25493)
304+
305+
// negative
306+
checkHiveHashForIntervalType("interval -1 minute", 25456)
307+
308+
// edge / boundary cases
309+
checkHiveHashForIntervalType("interval 0 minute", 23273)
310+
checkHiveHashForIntervalType("interval 2147483647 minute", 21830)
311+
checkHiveHashForIntervalType("interval -2147483648 minute", 22163)
312+
313+
// Out of range for both Hive and Spark
314+
// Hive throws an exception. Spark overflows and returns wrong output
315+
// checkHiveHashForIntervalType("interval 9999999999 minute", 0)
316+
317+
// ----- HOUR -----
318+
319+
// basic case
320+
checkHiveHashForIntervalType("interval 1 hour", 156473)
321+
322+
// negative
323+
checkHiveHashForIntervalType("interval -1 hour", 156436)
324+
325+
// edge / boundary cases
326+
checkHiveHashForIntervalType("interval 0 hour", 23273)
327+
checkHiveHashForIntervalType("interval 2147483647 hour", -62308)
328+
checkHiveHashForIntervalType("interval -2147483648 hour", -43327)
329+
330+
// Out of range for both Hive and Spark
331+
// Hive throws an exception. Spark overflows and returns wrong output
332+
// checkHiveHashForIntervalType("interval 9999999999 hour", 0)
333+
334+
// ----- DAY -----
335+
336+
// basic cases
337+
checkHiveHashForIntervalType("interval 1 day", 3220073)
338+
339+
// negative
340+
checkHiveHashForIntervalType("interval -1 day", 3220036)
341+
342+
// edge / boundary cases
343+
checkHiveHashForIntervalType("interval 0 day", 23273)
344+
checkHiveHashForIntervalType("interval 106751991 day", -451506760)
345+
checkHiveHashForIntervalType("interval -106751991 day", -451514123)
346+
347+
// Hive supports `day` for a longer range but Spark's range is smaller
348+
// The check for range is done at the parser level so this does not fail in Spark
349+
// checkHiveHashForIntervalType("interval -2147483648 day", -1575127)
350+
// checkHiveHashForIntervalType("interval 2147483647 day", -4767228)
351+
352+
// Out of range for both Hive and Spark
353+
// Hive throws an exception. Spark overflows and returns wrong output
354+
// checkHiveHashForIntervalType("interval 9999999999 day", 0)
355+
356+
// ----- MIX -----
357+
358+
checkHiveHashForIntervalType("interval 0 day 0 hour", 23273)
359+
checkHiveHashForIntervalType("interval 0 day 0 hour 0 minute", 23273)
360+
checkHiveHashForIntervalType("interval 0 day 0 hour 0 minute 0 second", 23273)
361+
checkHiveHashForIntervalType("interval 0 day 0 hour 0 minute 0 second 0 millisecond", 23273)
362+
checkHiveHashForIntervalType(
363+
"interval 0 day 0 hour 0 minute 0 second 0 millisecond 0 microsecond", 23273)
364+
365+
checkHiveHashForIntervalType("interval 6 day 15 hour", 21202073)
366+
checkHiveHashForIntervalType("interval 5 day 4 hour 8 minute", 16557833)
367+
checkHiveHashForIntervalType("interval -23 day 56 hour -1111113 minute 9898989 second",
368+
-2128468593)
369+
checkHiveHashForIntervalType("interval 66 day 12 hour 39 minute 23 second 987 millisecond",
370+
1199697904)
371+
checkHiveHashForIntervalType(
372+
"interval 66 day 12 hour 39 minute 23 second 987 millisecond 123 microsecond", 1199820904)
373+
}
374+
171375
test("hive-hash for array") {
172376
// empty array
173377
checkHiveHash(

0 commit comments

Comments
 (0)