Skip to content

Commit e213023

Browse files
[Protocol, Spark] UTC normalize timestamp partition values (#3378)
## Description Currently, in the Delta Protocol, timestamps are not stored with their time zone. This leads to unexpected behavior when querying across systems with different timezones configured (e.g. different spark sessions for instance). For instance in Spark, the timestamp value will be adjusted to spark session time zone and written to the delta log partition values without TZ. If someone were to query the same "timestamp" from a different session timezone, the same time zone value it can fail to surface results due to partition pruning. What this change proposes to the delta lake protocol is to allow timestamp partition values to be adjusted to UTC and explicitly stored in partition values with a UTC suffix. The original approach is still supported for compatibility but it is recommended for newer writers to write with UTC suffix. This is also important for Iceberg Uniform conversion because Iceberg timestamps must be UTC adjusted. Now we have a well defined format for UTC in delta, we can convert string partition values to Iceberg longs to make Uniform conversion succeed. This change updates the Spark-Delta integration to write out the UTC adjusted values for timestamp types. This also addresses an issue of microsecond partitions where previously microsecond partitioning (not recommended but technically allowed) would not work and be truncated to seconds. ## How was this patch tested? Added unit tests for the following cases: 1.) UTC timestamp partition values round trip across different session TZ 2.) A delta log with a mix of Non-UTC and UTC partition values round trip across the same session TZ 3.) Timestamp No Timezone round trips across timezones (kind of a tautology but important to make sure that the timestamp_ntz does not get written with UTC timestamp unintentionally) 4.) Timestamp round trips across same session time zone: UTC normalized 5.) Timestamp round trips across same session time zone: session time normalized (this case worked before this change, so it's important that it keeps working after this change) Mix of microsecond/second level precision and dates before epoch (to test if everything works with negative) ## Does this PR introduce _any_ user-facing changes? Yes in the sense that new timestamp partition values will be the normalized UTC values.
1 parent 8f1b297 commit e213023

File tree

7 files changed

+316
-27
lines changed

7 files changed

+316
-27
lines changed

PROTOCOL.md

Lines changed: 6 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -1791,13 +1791,16 @@ Type | Serialization Format
17911791
string | No translation required
17921792
numeric types | The string representation of the number
17931793
date | Encoded as `{year}-{month}-{day}`. For example, `1970-01-01`
1794-
timestamp | Encoded as `{year}-{month}-{day} {hour}:{minute}:{second}` or `{year}-{month}-{day} {hour}:{minute}:{second}.{microsecond}` For example: `1970-01-01 00:00:00`, or `1970-01-01 00:00:00.123456`
1794+
timestamp | Encoded as `{year}-{month}-{day} {hour}:{minute}:{second}` or `{year}-{month}-{day} {hour}:{minute}:{second}.{microsecond}`. For example: `1970-01-01 00:00:00`, or `1970-01-01 00:00:00.123456`. Timestamps may also be encoded as an ISO8601 formatted timestamp adjusted to UTC timestamp such as `1970-01-01T00:00:00.123456Z`
17951795
timestamp without timezone | Encoded as `{year}-{month}-{day} {hour}:{minute}:{second}` or `{year}-{month}-{day} {hour}:{minute}:{second}.{microsecond}` For example: `1970-01-01 00:00:00`, or `1970-01-01 00:00:00.123456` To use this type, a table must support a feature `timestampNtz`. See section [Timestamp without timezone (TimestampNtz)](#timestamp-without-timezone-timestampNtz) for more information.
17961796
boolean | Encoded as the string "true" or "false"
17971797
binary | Encoded as a string of escaped binary values. For example, `"\u0001\u0002\u0003"`
17981798

1799-
Note: A `timestamp` value in a partition value doesn't store the time zone due to historical reasons.
1800-
It means its behavior looks similar to `timestamp without time zone` when it is used in a partition column.
1799+
Note: A timestamp value in a partition value may be stored in one of the following ways:
1800+
1. Without a timezone, where the timestamp should be interpreted using the time zone of the system which wrote to the table.
1801+
2. Adjusted to UTC and stored in ISO8601 format.
1802+
1803+
It is highly recommended that modern writers adjust the timestamp to UTC and store the timestamp in ISO8601 format as outlined in 2.
18011804

18021805
## Schema Serialization Format
18031806

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

Lines changed: 35 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -33,7 +33,9 @@ import org.apache.spark.internal.Logging
3333
import org.apache.spark.internal.io.FileCommitProtocol
3434
import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage
3535
import org.apache.spark.sql.catalyst.expressions.Cast
36-
import org.apache.spark.sql.types.StringType
36+
import org.apache.spark.sql.delta.files.DeltaFileFormatWriter.PartitionedTaskAttemptContextImpl
37+
import org.apache.spark.sql.delta.sources.DeltaSQLConf
38+
import org.apache.spark.sql.types.{DataType, StringType, TimestampType}
3739

3840
/**
3941
* Writes out the files to `path` and returns a list of them in `addedStatuses`. Includes
@@ -67,7 +69,7 @@ class DelayedCommitProtocol(
6769
// since there's no guarantee the stats will exist.
6870
@transient val addedStatuses = new ArrayBuffer[AddFile]
6971

70-
val timestampPartitionPattern = "yyyy-MM-dd HH:mm:ss[.S]"
72+
val timestampPartitionPattern = "yyyy-MM-dd HH:mm:ss[.SSSSSS][.S]"
7173

7274
// Constants for CDC partition manipulation. Used only in newTaskTempFile(), but we define them
7375
// here to avoid building a new redundant regex for every file.
@@ -123,23 +125,48 @@ class DelayedCommitProtocol(
123125
}
124126
}
125127

126-
protected def parsePartitions(dir: String): Map[String, String] = {
127-
// TODO: timezones?
128+
protected def parsePartitions(
129+
dir: String,
130+
taskContext: TaskAttemptContext): Map[String, String] = {
128131
// TODO: enable validatePartitionColumns?
132+
val useUtcNormalizedTimestamps = taskContext match {
133+
case _: PartitionedTaskAttemptContextImpl => taskContext.getConfiguration.getBoolean(
134+
DeltaSQLConf.UTC_TIMESTAMP_PARTITION_VALUES.key, true)
135+
case _ => false
136+
}
137+
138+
val partitionColumnToDataType: Map[String, DataType] =
139+
taskContext.asInstanceOf[PartitionedTaskAttemptContextImpl]
140+
.partitionColToDataType
141+
.filter(partitionCol => partitionCol._2 == TimestampType)
142+
129143
val dateFormatter = DateFormatter()
130144
val timestampFormatter =
131145
TimestampFormatter(timestampPartitionPattern, java.util.TimeZone.getDefault)
146+
147+
/**
148+
* ToDo: Remove the use of this PartitionUtils API with type inference logic
149+
* since the types are already known from the Delta metadata!
150+
*
151+
* Currently types are passed to the PartitionUtils.parsePartition API to facilitate
152+
* timestamp conversion to UTC. In all other cases, the type is just inferred as a String.
153+
* Note: the passed in timestampFormatter and timezone detail
154+
* is used for parsing from the string timestamp.
155+
* If utc normalization is enabled the parsed partition value will be adjusted to UTC
156+
* and output in iso8601 format.
157+
*/
132158
val parsedPartition =
133159
PartitionUtils
134160
.parsePartition(
135161
new Path(dir),
136162
typeInference = false,
137163
Set.empty,
138-
Map.empty,
164+
userSpecifiedDataTypes = partitionColumnToDataType,
139165
validatePartitionColumns = false,
140166
java.util.TimeZone.getDefault,
141167
dateFormatter,
142-
timestampFormatter)
168+
timestampFormatter,
169+
useUtcNormalizedTimestamps)
143170
._1
144171
.get
145172
parsedPartition
@@ -164,7 +191,8 @@ class DelayedCommitProtocol(
164191
*/
165192
override def newTaskTempFile(
166193
taskContext: TaskAttemptContext, dir: Option[String], ext: String): String = {
167-
val partitionValues = dir.map(parsePartitions).getOrElse(Map.empty[String, String])
194+
val partitionValues = dir.map(dir => parsePartitions(dir, taskContext))
195+
.getOrElse(Map.empty[String, String])
168196
val filename = getFileName(taskContext, ext, partitionValues)
169197
val relativePath = randomPrefixLength.map { prefixLength =>
170198
DeltaUtils.getRandomPrefix(prefixLength) // Generate a random prefix as a first choice

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

Lines changed: 20 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -44,6 +44,7 @@ import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec
4444
import org.apache.spark.sql.execution.datasources._
4545
import org.apache.spark.sql.execution.datasources.FileFormatWriter._
4646
import org.apache.spark.sql.internal.SQLConf
47+
import org.apache.spark.sql.types.DataType
4748
import org.apache.spark.util.{SerializableConfiguration, Utils}
4849

4950
/**
@@ -261,6 +262,8 @@ object DeltaFileFormatWriter extends LoggingShims {
261262

262263
val jobTrackerID = SparkHadoopWriterUtils.createJobTrackerID(new Date())
263264
val ret = new Array[WriteTaskResult](rddWithNonEmptyPartitions.partitions.length)
265+
val partitionColumnToDataType = description.partitionColumns
266+
.map(attr => (attr.name, attr.dataType)).toMap
264267
sparkSession.sparkContext.runJob(
265268
rddWithNonEmptyPartitions,
266269
(taskContext: TaskContext, iter: Iterator[InternalRow]) => {
@@ -272,7 +275,8 @@ object DeltaFileFormatWriter extends LoggingShims {
272275
sparkAttemptNumber = taskContext.taskAttemptId().toInt & Integer.MAX_VALUE,
273276
committer,
274277
iterator = iter,
275-
concurrentOutputWriterSpec = concurrentOutputWriterSpec
278+
concurrentOutputWriterSpec = concurrentOutputWriterSpec,
279+
partitionColumnToDataType
276280
)
277281
},
278282
rddWithNonEmptyPartitions.partitions.indices,
@@ -377,6 +381,14 @@ object DeltaFileFormatWriter extends LoggingShims {
377381
}
378382
}
379383

384+
class PartitionedTaskAttemptContextImpl(
385+
conf: Configuration,
386+
taskId: TaskAttemptID,
387+
partitionColumnToDataType: Map[String, DataType])
388+
extends TaskAttemptContextImpl(conf, taskId) {
389+
val partitionColToDataType: Map[String, DataType] = partitionColumnToDataType
390+
}
391+
380392
/** Writes data out in a single Spark task. */
381393
private def executeTask(
382394
description: WriteJobDescription,
@@ -386,7 +398,8 @@ object DeltaFileFormatWriter extends LoggingShims {
386398
sparkAttemptNumber: Int,
387399
committer: FileCommitProtocol,
388400
iterator: Iterator[InternalRow],
389-
concurrentOutputWriterSpec: Option[ConcurrentOutputWriterSpec]): WriteTaskResult = {
401+
concurrentOutputWriterSpec: Option[ConcurrentOutputWriterSpec],
402+
partitionColumnToDataType: Map[String, DataType]): WriteTaskResult = {
390403

391404
val jobId = SparkHadoopWriterUtils.createJobID(jobTrackerID, sparkStageId)
392405
val taskId = new TaskID(jobId, TaskType.MAP, sparkPartitionId)
@@ -402,7 +415,11 @@ object DeltaFileFormatWriter extends LoggingShims {
402415
hadoopConf.setBoolean("mapreduce.task.ismap", true)
403416
hadoopConf.setInt("mapreduce.task.partition", 0)
404417

405-
new TaskAttemptContextImpl(hadoopConf, taskAttemptId)
418+
if (partitionColumnToDataType.isEmpty) {
419+
new TaskAttemptContextImpl(hadoopConf, taskAttemptId)
420+
} else {
421+
new PartitionedTaskAttemptContextImpl(hadoopConf, taskAttemptId, partitionColumnToDataType)
422+
}
406423
}
407424

408425
committer.setupTask(taskAttemptContext)

spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala

Lines changed: 12 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2025,6 +2025,18 @@ trait DeltaSQLConfBase {
20252025
.doc("If true, post-commit hooks will by default throw an exception when they fail.")
20262026
.booleanConf
20272027
.createWithDefault(Utils.isTesting)
2028+
2029+
///////////
2030+
// UTC TIMESTAMP PARTITION VALUES
2031+
///////////////////
2032+
val UTC_TIMESTAMP_PARTITION_VALUES = buildConf("write.utcTimestampPartitionValues")
2033+
.internal()
2034+
.doc(
2035+
"""
2036+
| If true, write UTC normalized timestamp partition values to Delta Log.
2037+
|""".stripMargin)
2038+
.booleanConf
2039+
.createWithDefault(true)
20282040
}
20292041

20302042
object DeltaSQLConf extends DeltaSQLConfBase

spark/src/main/scala/org/apache/spark/sql/delta/util/PartitionUtils.scala

Lines changed: 28 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -40,6 +40,7 @@ package org.apache.spark.sql.delta.util
4040

4141
import java.lang.{Double => JDouble, Long => JLong}
4242
import java.math.{BigDecimal => JBigDecimal}
43+
import java.time.ZoneId
4344
import java.util.{Locale, TimeZone}
4445

4546
import scala.collection.mutable
@@ -48,6 +49,7 @@ import scala.util.Try
4849

4950
import org.apache.spark.sql.delta.{DeltaAnalysisException, DeltaErrors}
5051
import org.apache.hadoop.fs.Path
52+
import org.apache.spark.unsafe.types.UTF8String
5153

5254
// scalastyle:off import.ordering.noEmptyLine
5355
import org.apache.spark.sql.AnalysisException
@@ -117,7 +119,8 @@ object PartitionSpec {
117119

118120
private[delta] object PartitionUtils {
119121

120-
val timestampPartitionPattern = "yyyy-MM-dd HH:mm:ss[.S]"
122+
lazy val timestampPartitionPattern = "yyyy-MM-dd HH:mm:ss[.S]"
123+
lazy val utcFormatter = TimestampFormatter("yyyy-MM-dd'T'HH:mm:ss.SSSSSSz", ZoneId.of("Z"))
121124

122125
case class PartitionValues(columnNames: Seq[String], literals: Seq[Literal])
123126
{
@@ -279,7 +282,8 @@ private[delta] object PartitionUtils {
279282
validatePartitionColumns: Boolean,
280283
timeZone: TimeZone,
281284
dateFormatter: DateFormatter,
282-
timestampFormatter: TimestampFormatter): (Option[PartitionValues], Option[Path]) = {
285+
timestampFormatter: TimestampFormatter,
286+
useUtcNormalizedTimestamp: Boolean = false): (Option[PartitionValues], Option[Path]) = {
283287
val columns = ArrayBuffer.empty[(String, Literal)]
284288
// Old Hadoop versions don't have `Path.isRoot`
285289
var finished = path.getParent == null
@@ -301,7 +305,8 @@ private[delta] object PartitionUtils {
301305
// Once we get the string, we try to parse it and find the partition column and value.
302306
val maybeColumn =
303307
parsePartitionColumn(currentPath.getName, typeInference, userSpecifiedDataTypes,
304-
validatePartitionColumns, timeZone, dateFormatter, timestampFormatter)
308+
validatePartitionColumns, timeZone, dateFormatter, timestampFormatter,
309+
useUtcNormalizedTimestamp)
305310
maybeColumn.foreach(columns += _)
306311

307312
// Now, we determine if we should stop.
@@ -338,7 +343,8 @@ private[delta] object PartitionUtils {
338343
validatePartitionColumns: Boolean,
339344
timeZone: TimeZone,
340345
dateFormatter: DateFormatter,
341-
timestampFormatter: TimestampFormatter): Option[(String, Literal)] = {
346+
timestampFormatter: TimestampFormatter,
347+
useUtcNormalizedTimestamp: Boolean = false): Option[(String, Literal)] = {
342348
val equalSignIndex = columnSpec.indexOf('=')
343349
if (equalSignIndex == -1) {
344350
None
@@ -360,12 +366,25 @@ private[delta] object PartitionUtils {
360366
dateFormatter,
361367
timestampFormatter)
362368
val columnValue = columnValueLiteral.eval()
363-
val castedValue = Cast(columnValueLiteral, dataType, Option(timeZone.getID)).eval()
364-
if (validatePartitionColumns && columnValue != null && castedValue == null) {
365-
throw DeltaErrors.partitionColumnCastFailed(
366-
columnValue.toString, dataType.toString, columnName)
369+
if (dataType == DataTypes.TimestampType) {
370+
if (useUtcNormalizedTimestamp) {
371+
Try {
372+
Literal.create(
373+
utcFormatter.format(
374+
timestampFormatter.parse(columnValue.asInstanceOf[UTF8String].toString)),
375+
StringType)
376+
}.getOrElse(columnValueLiteral)
377+
} else {
378+
columnValueLiteral
379+
}
380+
} else {
381+
val castedValue = Cast(columnValueLiteral, dataType, Option(timeZone.getID)).eval()
382+
if (validatePartitionColumns && columnValue != null && castedValue == null) {
383+
throw DeltaErrors.partitionColumnCastFailed(
384+
columnValue.toString, dataType.toString, columnName)
385+
}
386+
Literal.create(castedValue, dataType)
367387
}
368-
Literal.create(castedValue, dataType)
369388
} else {
370389
inferPartitionColumnValue(
371390
rawColumnValue,

spark/src/main/scala/org/apache/spark/sql/delta/util/TimestampFormatter.scala

Lines changed: 9 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -67,15 +67,15 @@ sealed trait TimestampFormatter extends Serializable {
6767

6868
class Iso8601TimestampFormatter(
6969
pattern: String,
70-
timeZone: TimeZone,
70+
timeZone: ZoneId,
7171
locale: Locale) extends TimestampFormatter with DateTimeFormatterHelper {
7272
@transient
7373
protected lazy val formatter = getOrCreateFormatter(pattern, locale)
7474

7575
private def toInstant(s: String): Instant = {
7676
val temporalAccessor = formatter.parse(s)
7777
if (temporalAccessor.query(TemporalQueries.offset()) == null) {
78-
toInstantWithZoneId(temporalAccessor, timeZone.toZoneId)
78+
toInstantWithZoneId(temporalAccessor, timeZone)
7979
} else {
8080
Instant.from(temporalAccessor)
8181
}
@@ -85,7 +85,7 @@ class Iso8601TimestampFormatter(
8585

8686
override def format(us: Long): String = {
8787
val instant = DateTimeUtils.microsToInstant(us)
88-
formatter.withZone(timeZone.toZoneId).format(instant)
88+
formatter.withZone(timeZone).format(instant)
8989
}
9090
}
9191

@@ -98,7 +98,7 @@ class Iso8601TimestampFormatter(
9898
* @param timeZone the time zone in which the formatter parses or format timestamps
9999
*/
100100
class FractionTimestampFormatter(timeZone: TimeZone)
101-
extends Iso8601TimestampFormatter("", timeZone, TimestampFormatter.defaultLocale) {
101+
extends Iso8601TimestampFormatter("", timeZone.toZoneId, TimestampFormatter.defaultLocale) {
102102

103103
@transient
104104
override protected lazy val formatter = DateTimeFormatterHelper.fractionFormatter
@@ -108,8 +108,12 @@ object TimestampFormatter {
108108
val defaultPattern: String = "yyyy-MM-dd HH:mm:ss"
109109
val defaultLocale: Locale = Locale.US
110110

111+
def apply(format: String, zoneId: ZoneId): TimestampFormatter = {
112+
new Iso8601TimestampFormatter(format, zoneId, defaultLocale)
113+
}
114+
111115
def apply(format: String, timeZone: TimeZone, locale: Locale): TimestampFormatter = {
112-
new Iso8601TimestampFormatter(format, timeZone, locale)
116+
new Iso8601TimestampFormatter(format, timeZone.toZoneId, locale)
113117
}
114118

115119
def apply(format: String, timeZone: TimeZone): TimestampFormatter = {

0 commit comments

Comments
 (0)