Skip to content

Commit

Permalink
[SPARK-47473][SQL] Fix correctness issue of converting postgres INFIN…
Browse files Browse the repository at this point in the history
…ITY timestamps

This PR fixes a bug involved with #41843 that Epoch Second is used instead of epoch millis to create a timestamp value

bugfix

no

revised tests

no

Closes #45599 from yaooqinn/SPARK-47473.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Kent Yao <yao@apache.org>
(cherry picked from commit ad8ac17)
Signed-off-by: Kent Yao <yao@apache.org>
  • Loading branch information
yaooqinn committed Mar 20, 2024
1 parent 8fcd9a1 commit df2fddd
Show file tree
Hide file tree
Showing 2 changed files with 7 additions and 7 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -20,13 +20,13 @@ package org.apache.spark.sql.jdbc
import java.math.{BigDecimal => JBigDecimal}
import java.sql.{Connection, Date, Timestamp}
import java.text.SimpleDateFormat
import java.time.{LocalDateTime, ZoneOffset}
import java.time.LocalDateTime
import java.util.Properties

import org.apache.spark.sql.Column
import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.expressions.Literal
import org.apache.spark.sql.types.{ArrayType, DecimalType, FloatType, ShortType}
import org.apache.spark.sql.types._
import org.apache.spark.tags.DockerTest

/**
Expand Down Expand Up @@ -445,9 +445,8 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite {
assert(row.length == 2)
val infinity = row(0).getAs[Timestamp]("timestamp_column")
val negativeInfinity = row(1).getAs[Timestamp]("timestamp_column")
val minTimeStamp = LocalDateTime.of(1, 1, 1, 0, 0, 0).toEpochSecond(ZoneOffset.UTC)
val maxTimestamp = LocalDateTime.of(9999, 12, 31, 23, 59, 59).toEpochSecond(ZoneOffset.UTC)

val minTimeStamp = -62135596800000L
val maxTimestamp = 253402300799999L
assert(infinity.getTime == maxTimestamp)
assert(negativeInfinity.getTime == minTimeStamp)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -293,8 +293,9 @@ private object PostgresDialect extends JdbcDialect with SQLConfHelper {
val POSTGRESQL_DATE_POSITIVE_INFINITY = 9223372036825200000L
val POSTGRESQL_DATE_DATE_POSITIVE_SMALLER_INFINITY = 185543533774800000L

val minTimeStamp = LocalDateTime.of(1, 1, 1, 0, 0, 0).toEpochSecond(ZoneOffset.UTC)
val maxTimestamp = LocalDateTime.of(9999, 12, 31, 23, 59, 59).toEpochSecond(ZoneOffset.UTC)
val minTimeStamp = LocalDateTime.of(1, 1, 1, 0, 0, 0).toInstant(ZoneOffset.UTC).toEpochMilli
val maxTimestamp =
LocalDateTime.of(9999, 12, 31, 23, 59, 59, 999999999).toInstant(ZoneOffset.UTC).toEpochMilli

val time = t.getTime
if (time == POSTGRESQL_DATE_POSITIVE_INFINITY ||
Expand Down

0 comments on commit df2fddd

Please sign in to comment.