Skip to content

Commit 0d87908

Browse files
authored
[Spark] Fix CDC Commit Timestamp value under different Timezones (#3347)
<!-- 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'. -->
1 parent 70bfe82 commit 0d87908

File tree

5 files changed

+128
-54
lines changed

5 files changed

+128
-54
lines changed

spark/src/main/scala/org/apache/spark/sql/delta/files/CdcAddFileIndex.scala

+3
Original file line numberDiff line numberDiff line change
@@ -39,6 +39,9 @@ import org.apache.spark.sql.types.StructType
3939
* @param path The table's data path.
4040
* @param snapshot The snapshot where we read CDC from.
4141
* @param rowIndexFilters Map from <b>URI-encoded</b> file path to a row index filter type.
42+
*
43+
* Note: Please also consider other CDC-related file indexes like [[TahoeChangeFileIndex]]
44+
* and [[TahoeRemoveFileIndex]] when modifying this file index.
4245
*/
4346
class CdcAddFileIndex(
4447
spark: SparkSession,

spark/src/main/scala/org/apache/spark/sql/delta/files/TahoeChangeFileIndex.scala

+8-1
Original file line numberDiff line numberDiff line change
@@ -16,6 +16,8 @@
1616

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

19+
import java.text.SimpleDateFormat
20+
1921
import org.apache.spark.sql.delta.{DeltaLog, Snapshot, SnapshotDescriptor}
2022
import org.apache.spark.sql.delta.actions.{AddCDCFile, AddFile}
2123
import org.apache.spark.sql.delta.commands.cdc.CDCReader.{CDC_COMMIT_TIMESTAMP, CDC_COMMIT_VERSION, CDCDataSpec}
@@ -29,6 +31,9 @@ import org.apache.spark.sql.types.{LongType, StructType, TimestampType}
2931
/**
3032
* A [[TahoeFileIndex]] for scanning a sequence of CDC files. Similar to [[TahoeBatchFileIndex]],
3133
* the equivalent for reading [[AddFile]] actions.
34+
*
35+
* Note: Please also consider other CDC-related file indexes like [[CdcAddFileIndex]]
36+
* and [[TahoeRemoveFileIndex]] when modifying this file index.
3237
*/
3338
class TahoeChangeFileIndex(
3439
spark: SparkSession,
@@ -47,9 +52,11 @@ class TahoeChangeFileIndex(
4752
files.map { f =>
4853
// We add the metadata as faked partition columns in order to attach it on a per-file
4954
// basis.
55+
val tsOpt = Option(ts)
56+
.map(new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS Z").format(_)).orNull
5057
val newPartitionVals = f.partitionValues +
5158
(CDC_COMMIT_VERSION -> version.toString) +
52-
(CDC_COMMIT_TIMESTAMP -> Option(ts).map(_.toString).orNull)
59+
(CDC_COMMIT_TIMESTAMP -> tsOpt)
5360
AddFile(f.path, newPartitionVals, f.size, 0, dataChange = false, tags = f.tags)
5461
}
5562
}

spark/src/main/scala/org/apache/spark/sql/delta/files/TahoeRemoveFileIndex.scala

+8-1
Original file line numberDiff line numberDiff line change
@@ -16,6 +16,8 @@
1616

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

19+
import java.text.SimpleDateFormat
20+
1921
import org.apache.spark.sql.delta._
2022
import org.apache.spark.sql.delta.actions.{AddFile, RemoveFile}
2123
import org.apache.spark.sql.delta.commands.cdc.CDCReader
@@ -36,6 +38,9 @@ import org.apache.spark.sql.types.StructType
3638
* @param path The table's data path.
3739
* @param snapshot The snapshot where we read CDC from.
3840
* @param rowIndexFilters Map from <b>URI-encoded</b> file path to a row index filter type.
41+
*
42+
* Note: Please also consider other CDC-related file indexes like [[CdcAddFileIndex]]
43+
* and [[TahoeChangeFileIndex]] when modifying this file index.
3944
*/
4045
class TahoeRemoveFileIndex(
4146
spark: SparkSession,
@@ -62,9 +67,11 @@ class TahoeRemoveFileIndex(
6267
}
6368
// We add the metadata as faked partition columns in order to attach it on a per-file
6469
// basis.
70+
val tsOpt = Option(ts)
71+
.map(new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS Z").format(_)).orNull
6572
val newPartitionVals = r.partitionValues +
6673
(CDC_COMMIT_VERSION -> version.toString) +
67-
(CDC_COMMIT_TIMESTAMP -> Option(ts).map(_.toString).orNull) +
74+
(CDC_COMMIT_TIMESTAMP -> tsOpt) +
6875
(CDC_TYPE_COLUMN_NAME -> CDC_TYPE_DELETE_STRING)
6976
AddFile(
7077
path = r.path,

spark/src/test/scala/org/apache/spark/sql/delta/DeltaCDCSQLSuite.scala

+39-35
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@ import org.apache.spark.sql.delta.test.DeltaTestImplicits._
2424

2525
import org.apache.spark.sql.{AnalysisException, DataFrame}
2626
import org.apache.spark.sql.catalyst.TableIdentifier
27+
import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._
2728
import org.apache.spark.sql.functions._
2829
import org.apache.spark.sql.types.LongType
2930

@@ -218,43 +219,46 @@ class DeltaCDCSQLSuite extends DeltaCDCSuiteBase with DeltaColumnMappingTestUtil
218219

219220
test("resolve expression for timestamp function") {
220221
val tbl = "tbl"
221-
withTable(tbl) {
222-
createTblWithThreeVersions(tblName = Some(tbl))
223-
224-
val deltaLog = DeltaLog.forTable(spark, TableIdentifier(tbl))
225-
226-
val currentTime = new Date().getTime
227-
modifyDeltaTimestamp(deltaLog, 0, currentTime - 100000)
228-
modifyDeltaTimestamp(deltaLog, 1, currentTime)
229-
modifyDeltaTimestamp(deltaLog, 2, currentTime + 100000)
230-
231-
val readDf = sql(s"SELECT * FROM table_changes('$tbl', 0, now())")
232-
checkCDCAnswer(
233-
DeltaLog.forTable(spark, TableIdentifier("tbl")),
234-
readDf,
235-
spark.range(20)
236-
.withColumn("_change_type", lit("insert"))
237-
.withColumn("_commit_version", (col("id") / 10).cast(LongType))
222+
withDefaultTimeZone(UTC) {
223+
withTable(tbl) {
224+
createTblWithThreeVersions(tblName = Some(tbl))
225+
226+
val deltaLog = DeltaLog.forTable(spark, TableIdentifier(tbl))
227+
228+
val currentTime = new Date().getTime
229+
modifyDeltaTimestamp(deltaLog, 0, currentTime - 100000)
230+
modifyDeltaTimestamp(deltaLog, 1, currentTime)
231+
modifyDeltaTimestamp(deltaLog, 2, currentTime + 100000)
232+
233+
val readDf = sql(s"SELECT * FROM table_changes('$tbl', 0, now())")
234+
checkCDCAnswer(
235+
DeltaLog.forTable(spark, TableIdentifier("tbl")),
236+
readDf,
237+
spark.range(20)
238+
.withColumn("_change_type", lit("insert"))
239+
.withColumn("_commit_version", (col("id") / 10).cast(LongType))
238240
)
239241

240-
// more complex expression
241-
val readDf2 = sql(s"SELECT * FROM table_changes('$tbl', 0, now() + interval 5 seconds)")
242-
checkCDCAnswer(
243-
DeltaLog.forTable(spark, TableIdentifier("tbl")),
244-
readDf2,
245-
spark.range(20)
246-
.withColumn("_change_type", lit("insert"))
247-
.withColumn("_commit_version", (col("id") / 10).cast(LongType))
248-
)
249-
val readDf3 = sql("SELECT * FROM table_changes" +
250-
s"('$tbl', string(date_sub(current_date(), 1)), string(now()))")
251-
checkCDCAnswer(
252-
DeltaLog.forTable(spark, TableIdentifier("tbl")),
253-
readDf2,
254-
spark.range(20)
255-
.withColumn("_change_type", lit("insert"))
256-
.withColumn("_commit_version", (col("id") / 10).cast(LongType))
257-
)
242+
// more complex expression
243+
val readDf2 = sql(s"SELECT * FROM table_changes('$tbl', 0, now() + interval 5 seconds)")
244+
checkCDCAnswer(
245+
DeltaLog.forTable(spark, TableIdentifier("tbl")),
246+
readDf2,
247+
spark.range(20)
248+
.withColumn("_change_type", lit("insert"))
249+
.withColumn("_commit_version", (col("id") / 10).cast(LongType))
250+
)
251+
252+
val readDf3 = sql("SELECT * FROM table_changes" +
253+
s"('$tbl', string(date_sub(current_date(), 1)), string(now()))")
254+
checkCDCAnswer(
255+
DeltaLog.forTable(spark, TableIdentifier("tbl")),
256+
readDf3,
257+
spark.range(20)
258+
.withColumn("_change_type", lit("insert"))
259+
.withColumn("_commit_version", (col("id") / 10).cast(LongType))
260+
)
261+
}
258262
}
259263
}
260264

spark/src/test/scala/org/apache/spark/sql/delta/DeltaCDCSuite.scala

+70-17
Original file line numberDiff line numberDiff line change
@@ -34,7 +34,8 @@ import org.apache.spark.sql.delta.util.FileNames
3434
import org.apache.spark.SparkConf
3535
import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row}
3636
import org.apache.spark.sql.catalyst.TableIdentifier
37-
import org.apache.spark.sql.functions.{col, current_timestamp, floor, lit}
37+
import org.apache.spark.sql.functions.{col, current_timestamp, floor, lit, unix_timestamp}
38+
import org.apache.spark.sql.internal.SQLConf
3839
import org.apache.spark.sql.streaming.StreamingQueryException
3940
import org.apache.spark.sql.test.SharedSparkSession
4041
import org.apache.spark.sql.types.{LongType, StringType, StructType}
@@ -62,7 +63,7 @@ abstract class DeltaCDCSuiteBase
6263
case class EndingVersion(value: String) extends Boundary
6364
case class EndingTimestamp(value: String) extends Boundary
6465
case object Unbounded extends Boundary // used to model situation when a boundary isn't provided
65-
val dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss")
66+
val dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss Z")
6667

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

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

261260
modifyDeltaTimestamp(deltaLog, 2, 2000)
262261

@@ -447,8 +446,7 @@ abstract class DeltaCDCSuiteBase
447446

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

454452
// Verify DST is respected.
@@ -477,6 +475,66 @@ abstract class DeltaCDCSuiteBase
477475
}
478476
}
479477

478+
test("CDC read's commit timestamps are correct under different timezones") {
479+
val tblName = "tbl"
480+
withTable(tblName) {
481+
spark.sql(s"CREATE OR REPLACE TABLE $tblName(id INT, name STRING, age INT) " +
482+
s"USING DELTA TBLPROPERTIES (delta.enableChangeDataFeed = true)")
483+
spark.sql(s"INSERT INTO $tblName(id, name, age) VALUES (1,'abc',20)")
484+
spark.sql(s"INSERT INTO $tblName(id, name, age) VALUES (2,'def',21)")
485+
spark.sql(s"UPDATE $tblName SET age = 19 WHERE id = 1")
486+
spark.sql(s"INSERT INTO $tblName(id, name, age) VALUES (3,'ghi',15)")
487+
spark.sql(s"DELETE FROM $tblName WHERE id = 3")
488+
489+
// unix_timestamp() on a Timestamp column returns the UNIX timestamp of the specified
490+
// time under the given SESSION_LOCAL_TIMEZONE, while collect() on a timestamp column
491+
// always returns the Timestamp in UTC.
492+
// By using unix_timestamp() on the commit timestamp column, we can accurately determine
493+
// whether or not the timestamp under different timezones represent the same point in time.
494+
val startingVersion = StartingVersion("0")
495+
val endingVersion = EndingVersion("10")
496+
spark.conf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, "America/Chicago")
497+
val readDfChicago = cdcRead(new TableName(tblName), startingVersion, endingVersion)
498+
.orderBy(CDC_COMMIT_VERSION, CDC_TYPE_COLUMN_NAME)
499+
.select(col(CDC_COMMIT_VERSION), col(CDC_TYPE_COLUMN_NAME),
500+
unix_timestamp(col(CDC_COMMIT_TIMESTAMP)))
501+
val readDfChicagoRows = readDfChicago.collect()
502+
503+
spark.conf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, "Asia/Ho_Chi_Minh")
504+
val readDfHCM = cdcRead(new TableName(tblName), startingVersion, endingVersion)
505+
.orderBy(CDC_COMMIT_VERSION, CDC_TYPE_COLUMN_NAME)
506+
.select(col(CDC_COMMIT_VERSION), col(CDC_TYPE_COLUMN_NAME),
507+
unix_timestamp(col(CDC_COMMIT_TIMESTAMP)))
508+
val readDfHCMRows = readDfHCM.collect()
509+
510+
spark.conf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, "UTC")
511+
val readDfUTC = cdcRead(new TableName(tblName), startingVersion, endingVersion)
512+
.orderBy(CDC_COMMIT_VERSION, CDC_TYPE_COLUMN_NAME)
513+
.select(col(CDC_COMMIT_VERSION), col(CDC_TYPE_COLUMN_NAME),
514+
unix_timestamp(col(CDC_COMMIT_TIMESTAMP)))
515+
val readDfUTCRows = readDfUTC.collect()
516+
517+
def checkCDCTimestampEqual(firstRows: Array[Row], secondRows: Array[Row]): Boolean = {
518+
assert(firstRows.length === secondRows.length,
519+
"Number of rows from 2 DFs should be the same.")
520+
for ((firstRow, secondRow) <- firstRows.zip(secondRows)) {
521+
assert(firstRow.getLong(0) === secondRow.getLong(0),
522+
"Commit version should be the same for every rows.")
523+
assert(firstRow.getString(1) === secondRow.getString(1),
524+
"Change type should be the same for every rows.")
525+
if (firstRow.getLong(2) != secondRow.getLong(2)) {
526+
return false
527+
}
528+
}
529+
true
530+
}
531+
532+
assert(checkCDCTimestampEqual(readDfChicagoRows, readDfHCMRows) === true)
533+
assert(checkCDCTimestampEqual(readDfChicagoRows, readDfUTCRows) === true)
534+
assert(checkCDCTimestampEqual(readDfHCMRows, readDfUTCRows) === true)
535+
}
536+
}
537+
480538
test("start version is provided and no end version") {
481539
val tblName = "tbl"
482540
withTable(tblName) {
@@ -742,11 +800,8 @@ abstract class DeltaCDCSuiteBase
742800

743801
// version 2
744802
modifyDeltaTimestamp(deltaLog, 2, 2000)
745-
746-
val tsStart = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss")
747-
.format(new Date(3000))
748-
val tsEnd = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss")
749-
.format(new Date(4000))
803+
val tsStart = dateFormat.format(new Date(3000))
804+
val tsEnd = dateFormat.format(new Date(4000))
750805

751806
val readDf = cdcRead(
752807
new TablePath(path),
@@ -776,10 +831,8 @@ abstract class DeltaCDCSuiteBase
776831
// version 2
777832
modifyDeltaTimestamp(deltaLog, 2, 2000)
778833

779-
val tsStart = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss")
780-
.format(new Date(0))
781-
val tsEnd = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss")
782-
.format(new Date(4000))
834+
val tsStart = dateFormat.format(new Date(0))
835+
val tsEnd = dateFormat.format(new Date(4000))
783836

784837
val readDf = cdcRead(
785838
new TablePath(tempDir.getAbsolutePath),

0 commit comments

Comments
 (0)