Skip to content

Commit

Permalink
[Spark] Fix CDC Commit Timestamp value under different Timezones (#3347)
Browse files Browse the repository at this point in the history
<!--
Thanks for sending a pull request!  Here are some tips for you:
1. If this is your first time, please read our contributor guidelines:
https://github.com/delta-io/delta/blob/master/CONTRIBUTING.md
2. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP]
Your PR title ...'.
  3. Be sure to keep the PR description updated to reflect all changes.
  4. Please write your PR title to summarize what this PR proposes.
5. If possible, provide a concise example to reproduce the issue for a
faster review.
6. If applicable, include the corresponding issue number in the PR title
and link it in the body.
-->

#### Which Delta project/connector is this regarding?
<!--
Please add the component selected below to the beginning of the pull
request title
For example: [Spark] Title of my pull request
-->

- [X] Spark
- [ ] Standalone
- [ ] Flink
- [ ] Kernel
- [ ] Other (fill in here)

## Description

<!--
- Describe what this PR changes.
- Describe why we need the change.
 
If this PR resolves an issue be sure to include "Resolves #XXX" to
correctly link and close the issue upon merge.
-->
The CDC's `_commit_timestamp` is incorrect when we try to read/display
it under a different Spark Session's timezone
`spark.sql.session.timeZone` (e.g. `America/Chicago`,
`Asia/Ho_Chi_Minh`, ...).

In this PR, we address this issue by taking into account timezone to
capture the precise point in time when we convert `CDCDataSpec`'s [Java
Timestamp](https://github.com/delta-io/delta/blob/master/spark/src/main/scala/org/apache/spark/sql/delta/commands/cdc/CDCReader.scala#L173)
field to [Spark's
Timestamp](https://github.com/delta-io/delta/blob/master/spark/src/main/scala/org/apache/spark/sql/delta/commands/cdc/CDCReader.scala#L80)
for the `_commit_timestamp` column for all CDC's file indexes
(`CDCAddFileIndex`, `TahoeRemoveFileIndex`, `TahoeChangeFileIndex`).

This is needed in order for CDF to work properly under a different
timezone than `UTC`.

## How was this patch tested?
Added UT, some minor UTs fix to take into account timezone.
<!--
If tests were added, say they were added here. Please make sure to test
the changes thoroughly including negative and positive cases if
possible.
If the changes were tested in any way other than unit tests, please
clarify how you tested step by step (ideally copy and paste-able, so
that other reviewers can test and check, and descendants can verify in
the future).
If the changes were not tested, please explain why.
-->

## Does this PR introduce _any_ user-facing changes?
Yes.

- CDC's `_commit_timestamp` should now be correct when we try to
read/display it under a different Spark Session's timezone
`spark.sql.session.timeZone` (e.g.` America/Chicago`,
`Asia/Ho_Chi_Minh`, ...).
- This is a user-facing change compared to the released Delta Lake
versions and within the unreleased branches such as master.

<!--
If yes, please clarify the previous behavior and the change this PR
proposes - provide the console output, description and/or an example to
show the behavior difference if possible.
If possible, please also clarify if this is a user-facing change
compared to the released Delta Lake versions or within the unreleased
branches such as master.
If no, write 'No'.
-->
  • Loading branch information
longvu-db authored Jul 9, 2024
1 parent 70bfe82 commit 0d87908
Show file tree
Hide file tree
Showing 5 changed files with 128 additions and 54 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,9 @@ import org.apache.spark.sql.types.StructType
* @param path The table's data path.
* @param snapshot The snapshot where we read CDC from.
* @param rowIndexFilters Map from <b>URI-encoded</b> file path to a row index filter type.
*
* Note: Please also consider other CDC-related file indexes like [[TahoeChangeFileIndex]]
* and [[TahoeRemoveFileIndex]] when modifying this file index.
*/
class CdcAddFileIndex(
spark: SparkSession,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,8 @@

package org.apache.spark.sql.delta.files

import java.text.SimpleDateFormat

import org.apache.spark.sql.delta.{DeltaLog, Snapshot, SnapshotDescriptor}
import org.apache.spark.sql.delta.actions.{AddCDCFile, AddFile}
import org.apache.spark.sql.delta.commands.cdc.CDCReader.{CDC_COMMIT_TIMESTAMP, CDC_COMMIT_VERSION, CDCDataSpec}
Expand All @@ -29,6 +31,9 @@ import org.apache.spark.sql.types.{LongType, StructType, TimestampType}
/**
* A [[TahoeFileIndex]] for scanning a sequence of CDC files. Similar to [[TahoeBatchFileIndex]],
* the equivalent for reading [[AddFile]] actions.
*
* Note: Please also consider other CDC-related file indexes like [[CdcAddFileIndex]]
* and [[TahoeRemoveFileIndex]] when modifying this file index.
*/
class TahoeChangeFileIndex(
spark: SparkSession,
Expand All @@ -47,9 +52,11 @@ class TahoeChangeFileIndex(
files.map { f =>
// We add the metadata as faked partition columns in order to attach it on a per-file
// basis.
val tsOpt = Option(ts)
.map(new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS Z").format(_)).orNull
val newPartitionVals = f.partitionValues +
(CDC_COMMIT_VERSION -> version.toString) +
(CDC_COMMIT_TIMESTAMP -> Option(ts).map(_.toString).orNull)
(CDC_COMMIT_TIMESTAMP -> tsOpt)
AddFile(f.path, newPartitionVals, f.size, 0, dataChange = false, tags = f.tags)
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,8 @@

package org.apache.spark.sql.delta.files

import java.text.SimpleDateFormat

import org.apache.spark.sql.delta._
import org.apache.spark.sql.delta.actions.{AddFile, RemoveFile}
import org.apache.spark.sql.delta.commands.cdc.CDCReader
Expand All @@ -36,6 +38,9 @@ import org.apache.spark.sql.types.StructType
* @param path The table's data path.
* @param snapshot The snapshot where we read CDC from.
* @param rowIndexFilters Map from <b>URI-encoded</b> file path to a row index filter type.
*
* Note: Please also consider other CDC-related file indexes like [[CdcAddFileIndex]]
* and [[TahoeChangeFileIndex]] when modifying this file index.
*/
class TahoeRemoveFileIndex(
spark: SparkSession,
Expand All @@ -62,9 +67,11 @@ class TahoeRemoveFileIndex(
}
// We add the metadata as faked partition columns in order to attach it on a per-file
// basis.
val tsOpt = Option(ts)
.map(new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS Z").format(_)).orNull
val newPartitionVals = r.partitionValues +
(CDC_COMMIT_VERSION -> version.toString) +
(CDC_COMMIT_TIMESTAMP -> Option(ts).map(_.toString).orNull) +
(CDC_COMMIT_TIMESTAMP -> tsOpt) +
(CDC_TYPE_COLUMN_NAME -> CDC_TYPE_DELETE_STRING)
AddFile(
path = r.path,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import org.apache.spark.sql.delta.test.DeltaTestImplicits._

import org.apache.spark.sql.{AnalysisException, DataFrame}
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._
import org.apache.spark.sql.functions._
import org.apache.spark.sql.types.LongType

Expand Down Expand Up @@ -218,43 +219,46 @@ class DeltaCDCSQLSuite extends DeltaCDCSuiteBase with DeltaColumnMappingTestUtil

test("resolve expression for timestamp function") {
val tbl = "tbl"
withTable(tbl) {
createTblWithThreeVersions(tblName = Some(tbl))

val deltaLog = DeltaLog.forTable(spark, TableIdentifier(tbl))

val currentTime = new Date().getTime
modifyDeltaTimestamp(deltaLog, 0, currentTime - 100000)
modifyDeltaTimestamp(deltaLog, 1, currentTime)
modifyDeltaTimestamp(deltaLog, 2, currentTime + 100000)

val readDf = sql(s"SELECT * FROM table_changes('$tbl', 0, now())")
checkCDCAnswer(
DeltaLog.forTable(spark, TableIdentifier("tbl")),
readDf,
spark.range(20)
.withColumn("_change_type", lit("insert"))
.withColumn("_commit_version", (col("id") / 10).cast(LongType))
withDefaultTimeZone(UTC) {
withTable(tbl) {
createTblWithThreeVersions(tblName = Some(tbl))

val deltaLog = DeltaLog.forTable(spark, TableIdentifier(tbl))

val currentTime = new Date().getTime
modifyDeltaTimestamp(deltaLog, 0, currentTime - 100000)
modifyDeltaTimestamp(deltaLog, 1, currentTime)
modifyDeltaTimestamp(deltaLog, 2, currentTime + 100000)

val readDf = sql(s"SELECT * FROM table_changes('$tbl', 0, now())")
checkCDCAnswer(
DeltaLog.forTable(spark, TableIdentifier("tbl")),
readDf,
spark.range(20)
.withColumn("_change_type", lit("insert"))
.withColumn("_commit_version", (col("id") / 10).cast(LongType))
)

// more complex expression
val readDf2 = sql(s"SELECT * FROM table_changes('$tbl', 0, now() + interval 5 seconds)")
checkCDCAnswer(
DeltaLog.forTable(spark, TableIdentifier("tbl")),
readDf2,
spark.range(20)
.withColumn("_change_type", lit("insert"))
.withColumn("_commit_version", (col("id") / 10).cast(LongType))
)
val readDf3 = sql("SELECT * FROM table_changes" +
s"('$tbl', string(date_sub(current_date(), 1)), string(now()))")
checkCDCAnswer(
DeltaLog.forTable(spark, TableIdentifier("tbl")),
readDf2,
spark.range(20)
.withColumn("_change_type", lit("insert"))
.withColumn("_commit_version", (col("id") / 10).cast(LongType))
)
// more complex expression
val readDf2 = sql(s"SELECT * FROM table_changes('$tbl', 0, now() + interval 5 seconds)")
checkCDCAnswer(
DeltaLog.forTable(spark, TableIdentifier("tbl")),
readDf2,
spark.range(20)
.withColumn("_change_type", lit("insert"))
.withColumn("_commit_version", (col("id") / 10).cast(LongType))
)

val readDf3 = sql("SELECT * FROM table_changes" +
s"('$tbl', string(date_sub(current_date(), 1)), string(now()))")
checkCDCAnswer(
DeltaLog.forTable(spark, TableIdentifier("tbl")),
readDf3,
spark.range(20)
.withColumn("_change_type", lit("insert"))
.withColumn("_commit_version", (col("id") / 10).cast(LongType))
)
}
}
}

Expand Down
87 changes: 70 additions & 17 deletions spark/src/test/scala/org/apache/spark/sql/delta/DeltaCDCSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,8 @@ import org.apache.spark.sql.delta.util.FileNames
import org.apache.spark.SparkConf
import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row}
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.functions.{col, current_timestamp, floor, lit}
import org.apache.spark.sql.functions.{col, current_timestamp, floor, lit, unix_timestamp}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.streaming.StreamingQueryException
import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.{LongType, StringType, StructType}
Expand Down Expand Up @@ -62,7 +63,7 @@ abstract class DeltaCDCSuiteBase
case class EndingVersion(value: String) extends Boundary
case class EndingTimestamp(value: String) extends Boundary
case object Unbounded extends Boundary // used to model situation when a boundary isn't provided
val dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss")
val dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss Z")

def createTblWithThreeVersions(
tblName: Option[String] = None,
Expand Down Expand Up @@ -250,13 +251,11 @@ abstract class DeltaCDCSuiteBase
// modify timestamps
// version 0
modifyDeltaTimestamp(deltaLog, 0, 0)
val tsAfterV0 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss")
.format(new Date(1))
val tsAfterV0 = dateFormat.format(new Date(1))

// version 1
modifyDeltaTimestamp(deltaLog, 1, 1000)
val tsAfterV1 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss")
.format(new Date(1001))
val tsAfterV1 = dateFormat.format(new Date(1001))

modifyDeltaTimestamp(deltaLog, 2, 2000)

Expand Down Expand Up @@ -447,8 +446,7 @@ abstract class DeltaCDCSuiteBase

// Set commit time during Daylight savings time change.
val restoreDate = "2022-11-06 01:42:44"
val format = new java.text.SimpleDateFormat("yyyy-MM-dd hh:mm:ss Z")
val timestamp = format.parse(s"$restoreDate -0800").getTime
val timestamp = dateFormat.parse(s"$restoreDate -0800").getTime
modifyDeltaTimestamp(deltaLog, 0, timestamp)

// Verify DST is respected.
Expand Down Expand Up @@ -477,6 +475,66 @@ abstract class DeltaCDCSuiteBase
}
}

test("CDC read's commit timestamps are correct under different timezones") {
val tblName = "tbl"
withTable(tblName) {
spark.sql(s"CREATE OR REPLACE TABLE $tblName(id INT, name STRING, age INT) " +
s"USING DELTA TBLPROPERTIES (delta.enableChangeDataFeed = true)")
spark.sql(s"INSERT INTO $tblName(id, name, age) VALUES (1,'abc',20)")
spark.sql(s"INSERT INTO $tblName(id, name, age) VALUES (2,'def',21)")
spark.sql(s"UPDATE $tblName SET age = 19 WHERE id = 1")
spark.sql(s"INSERT INTO $tblName(id, name, age) VALUES (3,'ghi',15)")
spark.sql(s"DELETE FROM $tblName WHERE id = 3")

// unix_timestamp() on a Timestamp column returns the UNIX timestamp of the specified
// time under the given SESSION_LOCAL_TIMEZONE, while collect() on a timestamp column
// always returns the Timestamp in UTC.
// By using unix_timestamp() on the commit timestamp column, we can accurately determine
// whether or not the timestamp under different timezones represent the same point in time.
val startingVersion = StartingVersion("0")
val endingVersion = EndingVersion("10")
spark.conf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, "America/Chicago")
val readDfChicago = cdcRead(new TableName(tblName), startingVersion, endingVersion)
.orderBy(CDC_COMMIT_VERSION, CDC_TYPE_COLUMN_NAME)
.select(col(CDC_COMMIT_VERSION), col(CDC_TYPE_COLUMN_NAME),
unix_timestamp(col(CDC_COMMIT_TIMESTAMP)))
val readDfChicagoRows = readDfChicago.collect()

spark.conf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, "Asia/Ho_Chi_Minh")
val readDfHCM = cdcRead(new TableName(tblName), startingVersion, endingVersion)
.orderBy(CDC_COMMIT_VERSION, CDC_TYPE_COLUMN_NAME)
.select(col(CDC_COMMIT_VERSION), col(CDC_TYPE_COLUMN_NAME),
unix_timestamp(col(CDC_COMMIT_TIMESTAMP)))
val readDfHCMRows = readDfHCM.collect()

spark.conf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, "UTC")
val readDfUTC = cdcRead(new TableName(tblName), startingVersion, endingVersion)
.orderBy(CDC_COMMIT_VERSION, CDC_TYPE_COLUMN_NAME)
.select(col(CDC_COMMIT_VERSION), col(CDC_TYPE_COLUMN_NAME),
unix_timestamp(col(CDC_COMMIT_TIMESTAMP)))
val readDfUTCRows = readDfUTC.collect()

def checkCDCTimestampEqual(firstRows: Array[Row], secondRows: Array[Row]): Boolean = {
assert(firstRows.length === secondRows.length,
"Number of rows from 2 DFs should be the same.")
for ((firstRow, secondRow) <- firstRows.zip(secondRows)) {
assert(firstRow.getLong(0) === secondRow.getLong(0),
"Commit version should be the same for every rows.")
assert(firstRow.getString(1) === secondRow.getString(1),
"Change type should be the same for every rows.")
if (firstRow.getLong(2) != secondRow.getLong(2)) {
return false
}
}
true
}

assert(checkCDCTimestampEqual(readDfChicagoRows, readDfHCMRows) === true)
assert(checkCDCTimestampEqual(readDfChicagoRows, readDfUTCRows) === true)
assert(checkCDCTimestampEqual(readDfHCMRows, readDfUTCRows) === true)
}
}

test("start version is provided and no end version") {
val tblName = "tbl"
withTable(tblName) {
Expand Down Expand Up @@ -742,11 +800,8 @@ abstract class DeltaCDCSuiteBase

// version 2
modifyDeltaTimestamp(deltaLog, 2, 2000)

val tsStart = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss")
.format(new Date(3000))
val tsEnd = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss")
.format(new Date(4000))
val tsStart = dateFormat.format(new Date(3000))
val tsEnd = dateFormat.format(new Date(4000))

val readDf = cdcRead(
new TablePath(path),
Expand Down Expand Up @@ -776,10 +831,8 @@ abstract class DeltaCDCSuiteBase
// version 2
modifyDeltaTimestamp(deltaLog, 2, 2000)

val tsStart = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss")
.format(new Date(0))
val tsEnd = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss")
.format(new Date(4000))
val tsStart = dateFormat.format(new Date(0))
val tsEnd = dateFormat.format(new Date(4000))

val readDf = cdcRead(
new TablePath(tempDir.getAbsolutePath),
Expand Down

0 comments on commit 0d87908

Please # to comment.