-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-12297] Table timezone correction for Timestamps #19250
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 3 commits
54f87c2
53b9fbe
5105b72
c5571a8
950d33a
e36851e
2069b65
74a9905
515b38b
a869c6e
fbab478
964af6d
0b4ba84
d5086c2
0b2aaef
63016db
7ee9c8a
db0d527
e74ce2d
1eaa045
5c03e07
5607160
7e44486
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -109,6 +109,11 @@ object DateTimeUtils { | |
| computedTimeZones.computeIfAbsent(timeZoneId, computeTimeZone) | ||
| } | ||
|
|
||
| lazy val validTimezones = TimeZone.getAvailableIDs().toSet | ||
| def isValidTimezone(timezoneId: String): Boolean = { | ||
| validTimezones.contains(timezoneId) | ||
| } | ||
|
|
||
| def newDateFormat(formatString: String, timeZone: TimeZone): DateFormat = { | ||
| val sdf = new SimpleDateFormat(formatString, Locale.US) | ||
| sdf.setTimeZone(timeZone) | ||
|
|
@@ -1015,6 +1020,10 @@ object DateTimeUtils { | |
| guess | ||
| } | ||
|
|
||
| def convertTz(ts: SQLTimestamp, fromZone: String, toZone: String): SQLTimestamp = { | ||
| convertTz(ts, getTimeZone(fromZone), getTimeZone(toZone)) | ||
|
||
| } | ||
|
|
||
| /** | ||
| * Convert the timestamp `ts` from one timezone to another. | ||
| * | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.catalog._ | |
| import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan} | ||
| import org.apache.spark.sql.execution.SQLExecution | ||
| import org.apache.spark.sql.execution.command.DDLUtils | ||
| import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, LogicalRelation} | ||
| import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, LogicalRelation, TimestampTableTimeZone} | ||
| import org.apache.spark.sql.sources.BaseRelation | ||
| import org.apache.spark.sql.types.StructType | ||
|
|
||
|
|
@@ -215,6 +215,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { | |
| */ | ||
| def save(path: String): Unit = { | ||
| this.extraOptions += ("path" -> path) | ||
| TimestampTableTimeZone.checkTableTz(s"for path $path", extraOptions.toMap) | ||
| save() | ||
| } | ||
|
|
||
|
|
@@ -266,6 +267,10 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { | |
| * @since 1.4.0 | ||
| */ | ||
| def insertInto(tableName: String): Unit = { | ||
| extraOptions.get(TimestampTableTimeZone.TIMEZONE_PROPERTY).foreach { tz => | ||
|
||
| throw new AnalysisException("Cannot provide a table timezone on insert; tried to insert " + | ||
| s"$tableName with ${TimestampTableTimeZone.TIMEZONE_PROPERTY}=$tz") | ||
| } | ||
| insertInto(df.sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName)) | ||
| } | ||
|
|
||
|
|
@@ -406,6 +411,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { | |
| } else { | ||
| CatalogTableType.MANAGED | ||
| } | ||
| val props = | ||
| extraOptions.filterKeys(key => key == TimestampTableTimeZone.TIMEZONE_PROPERTY).toMap | ||
| TimestampTableTimeZone.checkTableTz(tableIdent, props) | ||
|
|
||
| val tableDesc = CatalogTable( | ||
| identifier = tableIdent, | ||
|
|
@@ -414,7 +422,8 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { | |
| schema = new StructType, | ||
| provider = Some(source), | ||
| partitionColumnNames = partitioningColumns.getOrElse(Nil), | ||
| bucketSpec = getBucketSpec) | ||
| bucketSpec = getBucketSpec, | ||
| properties = props) | ||
|
|
||
| runCommand(df.sparkSession, "saveAsTable")(CreateTable(tableDesc, mode, Some(df.logicalPlan))) | ||
| } | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.catalog._ | |
| import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec | ||
| import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} | ||
| import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan | ||
| import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, PartitioningUtils} | ||
| import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, PartitioningUtils, TimestampTableTimeZone} | ||
| import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat | ||
| import org.apache.spark.sql.execution.datasources.parquet.ParquetSchemaConverter | ||
| import org.apache.spark.sql.internal.HiveSerDe | ||
|
|
@@ -230,6 +230,13 @@ case class AlterTableSetPropertiesCommand( | |
| isView: Boolean) | ||
| extends RunnableCommand { | ||
|
|
||
| if (isView) { | ||
| properties.get(TimestampTableTimeZone.TIMEZONE_PROPERTY).foreach { _ => | ||
|
||
| throw new AnalysisException("Timezone cannot be set for view") | ||
| } | ||
| } | ||
| TimestampTableTimeZone.checkTableTz(tableName, properties) | ||
|
|
||
| override def run(sparkSession: SparkSession): Seq[Row] = { | ||
| val catalog = sparkSession.sessionState.catalog | ||
| val table = catalog.getTableMetadata(tableName) | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
do we need a whole expression for this? can't we just reuse existing expressions? It's just simple arithmetics isn't it?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I guess you could use ToUTCTimestamp / FromUTCTimestamp for this, but that would be more expensive since you'd be doing the conversion twice for each value.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
What I'm saying is the analysis rule can just determine the delta, and then just do a simple add/delete.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Hmm, let me see if I can figure that out.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Unfortunately the offset depends on the actual date, so a timezone conversion can not be simplified to a simple delta.
Daylight saving time starts and ends on different days in different timezones, while some timezones don't have DST changes at all. Additionally, timezone rules have changed over time and keep changing. Both the basic timezone offset and the DST rules themselves could change (and have changed) over time.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ah, yes, that makes sense... it's also why my initial tests were failing at the DST boundaries. :-/