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

### What changes were proposed in this pull request?

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

### Why are the changes needed?

bugfix
### Does this PR introduce _any_ user-facing change?

no

### How was this patch tested?

revised tests
### Was this patch authored or co-authored using generative AI tooling?

no

Closes apache#45599 from yaooqinn/SPARK-47473.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Kent Yao <yao@apache.org>
  • Loading branch information
yaooqinn authored and sweisdb committed Apr 1, 2024
1 parent 1de8ca6 commit 4345dee
Show file tree
Hide file tree
Showing 2 changed files with 7 additions and 6 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -20,12 +20,12 @@ 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, Row}
import org.apache.spark.sql.catalyst.expressions.Literal
import org.apache.spark.sql.types.{ArrayType, DecimalType, FloatType, NullType, ShortType}
import org.apache.spark.sql.types._
import org.apache.spark.tags.DockerTest

/**
Expand Down Expand Up @@ -454,8 +454,8 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite {
val negativeInfinity = row(1).getAs[Timestamp]("timestamp_column")
val infinitySeq = row(0).getAs[scala.collection.Seq[Timestamp]]("timestamp_array")
val negativeInfinitySeq = row(1).getAs[scala.collection.Seq[Timestamp]]("timestamp_array")
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)
assert(infinitySeq.head.getTime == maxTimestamp)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -312,8 +312,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 4345dee

Please sign in to comment.