From 54f87c2c1e0ab0645fa5497553cf031f13e98c3b Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 28 Aug 2017 14:52:15 -0500 Subject: [PATCH 01/23] SPARK-12297. Table timezones. --- .../expressions/datetimeExpressions.scala | 67 +++++ .../sql/catalyst/util/DateTimeUtils.scala | 9 + .../expressions/DateExpressionsSuite.scala | 22 ++ .../apache/spark/sql/DataFrameWriter.scala | 13 +- .../execution/ParquetTimeZoneCorrection.scala | 212 ++++++++++++++ .../spark/sql/execution/command/ddl.scala | 5 +- .../spark/sql/execution/command/tables.scala | 7 +- .../parquet/ParquetFileFormat.scala | 24 +- .../sql/hive/HiveSessionStateBuilder.scala | 3 +- sql/hive/src/test/resources/log4j.properties | 21 +- .../hive/ParquetHiveCompatibilitySuite.scala | 261 +++++++++++++++++- 11 files changed, 634 insertions(+), 10 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/ParquetTimeZoneCorrection.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index 70354200c82df..cca7f5636a399 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -1212,6 +1212,73 @@ case class ToUTCTimestamp(left: Expression, right: Expression) } } +/** + * This modifies a timestamp to show how the display time changes going from one timezone to + * another, for the same instant in time. + * + * We intentionally do not provide an ExpressionDescription as this is not meant to be exposed to + * users, its only used for internal conversions. + */ +private[spark] case class ParquetTimestampCorrection( + time: Expression, + from: Expression, + to: Expression) + extends TernaryExpression with ImplicitCastInputTypes { + + // modeled on ToUTCTimestamp + Conv (as an example TernaryExpression) + + // convertTz() does the *opposite* conversion we want, which is why from & to appear reversed + // in all the calls to convertTz. Its used for showing how the display time changes when we go + // from one timezone to another. We want to see how should change the SQLTimestamp value to + // ensure the display does *not* change, despite going from one TZ to another. + + override def children: Seq[Expression] = Seq(time, from, to) + override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType, StringType, StringType) + override def dataType: DataType = TimestampType + override def prettyName: String = "parquet_timestamp_correction" + + override def nullSafeEval(time: Any, from: Any, to: Any): Any = { + DateTimeUtils.convertTz( + time.asInstanceOf[Long], + to.asInstanceOf[UTF8String].toString(), + from.asInstanceOf[UTF8String].toString()) + } + + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") + if (from.foldable && to.foldable) { + val fromTz = from.eval() + val toTz = to.eval() + if (fromTz == null || toTz == null) { + ev.copy(code = s""" + |boolean ${ev.isNull} = true; + |long ${ev.value} = 0; + """.stripMargin) + } else { + val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") + val eval = time.genCode(ctx) + ev.copy(code = s""" + |${eval.code} + |boolean ${ev.isNull} = ${eval.isNull}; + |long ${ev.value} = 0; + |if (!${ev.isNull}) { + | ${ev.value} = $dtu.convertTz(${eval.value}, "$toTz", "$fromTz"); + |} + """.stripMargin) + } + } else { + nullSafeCodeGen(ctx, ev, (time, from, to) => + s""" + |${ev.value} = $dtu.convertTz( + | $time, + | $to.toString(), + | $from.toString()); + """.stripMargin + ) + } + } +} + /** * Parses a column to a date based on the given format. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index 746c3e8950f7b..7f512b6042256 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -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. * diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index 89d99f9678cda..2538da6eed4ea 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -741,4 +741,26 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("2015-07-24 00:00:00", null, null) test(null, null, null) } + + test("parquet_timestamp_correction") { + def test(t: String, fromTz: String, toTz: String, expected: String): Unit = { + checkEvaluation( + ParquetTimestampCorrection( + Literal.create(if (t != null) Timestamp.valueOf(t) else null, TimestampType), + Literal.create(fromTz, StringType), + Literal.create(toTz, StringType)), + if (expected != null) Timestamp.valueOf(expected) else null) + checkEvaluation( + ParquetTimestampCorrection( + Literal.create(if (t != null) Timestamp.valueOf(t) else null, TimestampType), + NonFoldableLiteral.create(fromTz, StringType), + NonFoldableLiteral.create(toTz, StringType)), + if (expected != null) Timestamp.valueOf(expected) else null) + } + test("2015-07-24 00:00:00", "UTC", "PST", "2015-07-23 17:00:00") + test("2015-01-24 00:00:00", "UTC", "PST", "2015-01-23 16:00:00") + test(null, "UTC", "UTC", null) + test("2015-07-24 00:00:00", null, null, null) + test(null, null, null, null) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 07347d2748544..571cc683dab58 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -29,6 +29,7 @@ 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.parquet.ParquetFileFormat import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.types.StructType @@ -215,6 +216,8 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { */ def save(path: String): Unit = { this.extraOptions += ("path" -> path) + val destStr = s"for path $path" + ParquetFileFormat.checkTableTz(destStr, extraOptions.toMap) save() } @@ -266,6 +269,10 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * @since 1.4.0 */ def insertInto(tableName: String): Unit = { + extraOptions.get(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY).foreach { tz => + throw new AnalysisException("Cannot provide a table timezone on insert; tried to insert " + + s"$tableName with ${ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY}=$tz") + } insertInto(df.sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName)) } @@ -406,6 +413,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { } else { CatalogTableType.MANAGED } + val props = + extraOptions.filterKeys(key => key == ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY).toMap + ParquetFileFormat.checkTableTz(tableIdent, props) val tableDesc = CatalogTable( identifier = tableIdent, @@ -414,7 +424,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))) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ParquetTimeZoneCorrection.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ParquetTimeZoneCorrection.scala new file mode 100644 index 0000000000000..07a354d6c2065 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ParquetTimeZoneCorrection.scala @@ -0,0 +1,212 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution + +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.catalyst.catalog.CatalogTable +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.command.CreateDataSourceTableAsSelectCommand +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.types.{StringType, TimestampType} + +/** + * Apply a correction to data loaded from, or saved to, Parquet, so that it timestamps can be read + * like TIMESTAMP WITHOUT TIMEZONE. This gives correct behavior if you process data with + * machines in different timezones, or if you access the data from multiple SQL engines. + */ +case class ParquetTimeZoneCorrection(sparkSession: SparkSession) extends Rule[LogicalPlan] { + + def apply(plan: LogicalPlan): LogicalPlan = { + // we can't use transformUp because we want to terminate recursion if there was already + // timestamp correction, to keep this idempotent. + val r = plan match { + case insertIntoHadoopFs: InsertIntoHadoopFsRelationCommand => + if (!insertIntoHadoopFs.query.isInstanceOf[Project]) { + throw new AnalysisException("Expected the child of InsertIntoHadoopFsRelationCommand " + + s"to be a Project in $plan") + } + // The query might be reading from a parquet table which requires a different conversion; + // this makes sure we apply the correct conversions there. + val (fixedQuery, _) = readConversion(insertIntoHadoopFs.query) + + val tableTz = extractTableTz(insertIntoHadoopFs.catalogTable, insertIntoHadoopFs.options) + val x = tableTz.map{ tz => + writeConversion( + writer = insertIntoHadoopFs, + innerQuery = fixedQuery, + tableTz = tz, + copyWriter = insertIntoFsCopy(insertIntoHadoopFs, _) + ) + }.getOrElse(insertIntoHadoopFs) + logDebug(s"converted $plan to $x") + x + + case other => + // recurse into children to see if we're reading data that needs conversion + val (convertedPlan, _) = readConversion(plan) + convertedPlan + } + r + } + + private def readConversion( + plan: LogicalPlan + ): (LogicalPlan, Map[ExprId, NamedExpression]) = plan match { + case alreadyConverted@Project(exprs, _) if hasCorrection(exprs) => + logDebug(s"not transforming $alreadyConverted because its already converted") + (alreadyConverted, Map()) + case lr@LogicalRelation(fsRelation: HadoopFsRelation, _, _, _) => + val tzOpt = extractTableTz(lr.catalogTable, fsRelation.options) + tzOpt.map { tableTz => + // the table has a timezone set, so after reading the data, apply a conversion + + // SessionTZ will make the time display correctly in SQL queries, but incorrectly if you + // pull Timestamp objects out (eg. with a dataset.collect()) + val toTz = sparkSession.sessionState.conf.sessionLocalTimeZone + if (toTz != tableTz) { + logDebug(s"table tz = $tableTz; converting to current session tz = $toTz") + // find timestamp columns, and convert their tz + val (foundTs, modifiedFields, replacements) = + convertTzForAllTimestamps(lr, tableTz, toTz) + if (foundTs) { + (new Project(modifiedFields, lr), replacements) + } else { + (lr, Map[ExprId, NamedExpression]()) + } + } else { + (lr, Map[ExprId, NamedExpression]()) + } + }.getOrElse((lr, Map[ExprId, NamedExpression]())) + case other => + // first, process all the children -- this ensures we have the right renames in scope. + var newReplacements = Map[ExprId, NamedExpression]() + val fixedPlan = other.mapChildren { originalPlan => + val (newPlan, extraReplacements) = readConversion(originalPlan) + newReplacements ++= extraReplacements + newPlan + } + // now we need to adjust all names to use the new version. + val fixedExpressions = fixedPlan.mapExpressions { outerExp => + val adjustedExp = outerExp.transformUp { case exp: NamedExpression => + newReplacements.get(exp.exprId).getOrElse(exp) + } + logDebug(s"adjusted $outerExp to $adjustedExp using $newReplacements") + adjustedExp + } + (fixedExpressions, newReplacements) + } + + private def hasCorrection(exprs: Seq[NamedExpression]): Boolean = { + var hasCorrection = false + exprs.foreach { expr => + expr.foreach { + case _: ParquetTimestampCorrection => + hasCorrection = true + case other => // no-op + } + } + hasCorrection + } + + private def insertIntoFsCopy( + insert: InsertIntoHadoopFsRelationCommand, + newQuery: LogicalPlan): InsertIntoHadoopFsRelationCommand = { + insert.copy(query = newQuery) + } + + private def createDatasourceTableCopy( + insert: CreateDataSourceTableAsSelectCommand, + newQuery: LogicalPlan): CreateDataSourceTableAsSelectCommand = { + insert.copy(query = newQuery) + } + + private def writeConversion( + writer: LogicalPlan, + innerQuery: LogicalPlan, + tableTz: String, + copyWriter: LogicalPlan => LogicalPlan): LogicalPlan = { + val internalTz = sparkSession.sessionState.conf.sessionLocalTimeZone + if (tableTz != internalTz) { + val (foundTsFields, modifiedFields, _) = + convertTzForAllTimestamps(innerQuery, internalTz, tableTz) + if (foundTsFields) { + val adjustedFields = + new Project(modifiedFields, innerQuery) + val converted = copyWriter(adjustedFields) + converted + } else { + writer + } + } else { + writer + } + } + + private def extractTableTz(options: Map[String, String]): Option[String] = { + options.get(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY) + } + + private def extractTableTz( + table: Option[CatalogTable], + options: Map[String, String]): Option[String] = { + table.flatMap { tbl => extractTableTz(tbl.properties) }.orElse(extractTableTz(options)) + } + + /** + * Find all timestamp fields in the given relation. For each one, replace it with an expression + * that converts the timezone of the timestamp, and assigns an alias to that new expression. + * (Leave non-timestamp fields alone.) Also return a map from the original id for the timestamp + * field, to the new alias of the timezone-corrected expression. + */ + private def convertTzForAllTimestamps( + relation: LogicalPlan, + fromTz: String, + toTz: String): (Boolean, Seq[NamedExpression], Map[ExprId, NamedExpression]) = { + val schema = relation.schema + var foundTs = false + var replacements = Map[ExprId, NamedExpression]() + val modifiedFields: Seq[NamedExpression] = schema.map { field => + val exp = relation.resolve(Seq(field.name), sparkSession.sessionState.conf.resolver) + .getOrElse { + val inputColumns = schema.map(_.name).mkString(", ") + throw new AnalysisException( + s"cannot resolve '${field.name}' given input columns: [$inputColumns]") + } + if (field.dataType == TimestampType) { + foundTs = true + val adjustedTs = Alias( + ParquetTimestampCorrection( + exp, + Literal.create(fromTz, StringType), + Literal.create(toTz, StringType) + ), + field.name + )() + // we also need to rename all occurrences of this field further up in the plan + // to refer to our new adjusted timestamp, so we pass this replacement up the call stack. + replacements += exp.exprId -> adjustedTs.toAttribute + adjustedTs + } else { + exp + } + } + (foundTs, modifiedFields, replacements) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 162e1d5be2938..e2331c0b03ced 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -34,9 +34,10 @@ 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.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, PartitioningUtils} import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat -import org.apache.spark.sql.execution.datasources.parquet.ParquetSchemaConverter +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, ParquetSchemaConverter} import org.apache.spark.sql.internal.HiveSerDe import org.apache.spark.sql.types._ import org.apache.spark.util.{SerializableConfiguration, ThreadUtils} @@ -230,6 +231,8 @@ case class AlterTableSetPropertiesCommand( isView: Boolean) extends RunnableCommand { + ParquetFileFormat.checkTableTz(tableName, properties) + override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog val table = catalog.getTableMetadata(tableName) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 8d95ca6921cf8..ab367e7c49d0a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTableType._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} -import org.apache.spark.sql.catalyst.util.quoteIdentifier +import org.apache.spark.sql.catalyst.util.{quoteIdentifier, DateTimeUtils} import org.apache.spark.sql.execution.datasources.{DataSource, PartitioningUtils} import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat import org.apache.spark.sql.execution.datasources.json.JsonFileFormat @@ -86,7 +86,8 @@ case class CreateTableLikeCommand( schema = sourceTableDesc.schema, provider = newProvider, partitionColumnNames = sourceTableDesc.partitionColumnNames, - bucketSpec = sourceTableDesc.bucketSpec) + bucketSpec = sourceTableDesc.bucketSpec, + properties = sourceTableDesc.properties) catalog.createTable(newTableDesc, ifNotExists) Seq.empty[Row] @@ -126,6 +127,8 @@ case class CreateTableCommand( sparkSession.sessionState.catalog.createTable(table, ignoreIfExists) Seq.empty[Row] } + + ParquetFileFormat.checkTableTz(table.identifier, table.properties) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index e1e740500205a..64a124e883603 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -42,10 +42,11 @@ import org.apache.parquet.schema.MessageType import org.apache.spark.{SparkException, TaskContext} import org.apache.spark.internal.Logging import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.catalyst.parser.LegacyTypeStringParser +import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector import org.apache.spark.sql.internal.SQLConf @@ -645,4 +646,25 @@ object ParquetFileFormat extends Logging { Failure(cause) }.toOption } + + private[spark] val PARQUET_TIMEZONE_TABLE_PROPERTY = "parquet.mr.int96.write.zone" + + /** + * Throw an AnalysisException if we're trying to set an invalid timezone for this table. + */ + private[spark] def checkTableTz(table: TableIdentifier, properties: Map[String, String]): Unit = { + checkTableTz(s"in table ${table.toString}", properties) + } + + /** + * Throw an AnalysisException if we're trying to set an invalid timezone for this table. + */ + private[spark] def checkTableTz(dest: String, properties: Map[String, String]): Unit = { + properties.get(PARQUET_TIMEZONE_TABLE_PROPERTY).foreach { tz => + if (!DateTimeUtils.isValidTimezone(tz)) { + throw new AnalysisException(s"Cannot set $PARQUET_TIMEZONE_TABLE_PROPERTY to invalid " + + s"timezone $tz $dest") + } + } + } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala index 92cb4ef11c9e3..3df7faa16a825 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.Analyzer import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.SparkPlanner +import org.apache.spark.sql.execution.{ParquetTimeZoneCorrection, SparkPlanner} import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.hive.client.HiveClient import org.apache.spark.sql.internal.{BaseSessionStateBuilder, SessionResourceLoader, SessionState} @@ -80,6 +80,7 @@ class HiveSessionStateBuilder(session: SparkSession, parentState: Option[Session PreprocessTableInsertion(conf) +: DataSourceAnalysis(conf) +: HiveAnalysis +: + ParquetTimeZoneCorrection(session) +: customPostHocResolutionRules override val extendedCheckRules: Seq[LogicalPlan => Unit] = diff --git a/sql/hive/src/test/resources/log4j.properties b/sql/hive/src/test/resources/log4j.properties index a48ae9fc5edd8..c334d24a11b76 100644 --- a/sql/hive/src/test/resources/log4j.properties +++ b/sql/hive/src/test/resources/log4j.properties @@ -16,13 +16,19 @@ # # Set everything to be logged to the file core/target/unit-tests.log -log4j.rootLogger=DEBUG, CA, FA +log4j.rootLogger=WARN, CA, FA #Console Appender log4j.appender.CA=org.apache.log4j.ConsoleAppender log4j.appender.CA.layout=org.apache.log4j.PatternLayout log4j.appender.CA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %p %c: %m%n -log4j.appender.CA.Threshold = WARN +#log4j.appender.CA.Threshold = WARN +#log4j.logger=WARN, CA +#log4j.logger.org=WARN, CA +#log4j.logger.com=WARN, CA +#log4j.logger.io=WARN, CA +#log4j.logger.sun=WARN, CA +#log4j.logger.akka=WARN, CA #File Appender @@ -32,6 +38,17 @@ log4j.appender.FA.file=target/unit-tests.log log4j.appender.FA.layout=org.apache.log4j.PatternLayout log4j.appender.FA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# to get more logs, have to turn off Threshold line +# log4j.logger.org.apache.spark.sql.catalyst.expressions.codegen=DEBUG +# log4j.logger.org.apache.spark.sql.execution.WholeStageCodegenExec=DEBUG +# log4j.logger.org.apache.spark.sql.execution.ParquetTimeZoneCorrection=DEBUG +# analyzer +# log4j.logger.org.apache.spark.sql.hive.HiveSessionStateBuilder$$anon$1=TRACE +# optimizer +#log4j.logger.org.apache.spark.sql.internal.BaseSessionStateBuilder$$anon$2=TRACE + + # Set the logger level of File Appender to WARN log4j.appender.FA.Threshold = DEBUG diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala index 05b6059472f59..7b067845bcbae 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala @@ -17,12 +17,22 @@ package org.apache.spark.sql.hive +import java.io.File +import java.net.URLDecoder import java.sql.Timestamp +import java.util.TimeZone -import org.apache.spark.sql.Row -import org.apache.spark.sql.execution.datasources.parquet.ParquetCompatibilityTest +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.parquet.hadoop.ParquetFileReader +import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName + +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.execution.datasources.parquet.{ParquetCompatibilityTest, ParquetFileFormat} +import org.apache.spark.sql.functions._ import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{StringType, StructType, TimestampType} class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHiveSingleton { /** @@ -141,4 +151,251 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi Row(Seq(Row(1))), "ARRAY>") } + + // Check creating parquet tables with timestamps, writing data into them, and reading it back out + // under a variety of conditions: + // * tables with explicit tz and those without + // * altering table properties directly + // * variety of timezones, local & non-local + testCreateAlterTablesWithTimezone("no_tz", None, None) + testCreateAlterTablesWithTimezone("LA", Some("America/Los_Angeles"), None) + testCreateAlterTablesWithTimezone("LA", Some("Europe/Berlin"), None) + testCreateAlterTablesWithTimezone("LA", Some("Europe/Berlin"), Some("UTC")) + + private def checkHasTz(spark: SparkSession, table: String, tz: Option[String]): Unit = { + val tableMetadata = spark.sessionState.catalog.getTableMetadata(TableIdentifier(table)) + assert(tableMetadata.properties.get(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY) === tz, + s"for table $table") + } + + private def testCreateAlterTablesWithTimezone( + baseTable: String, + explicitTz: Option[String], + sessionTzOpt: Option[String]): Unit = { + test(s"SPARK-12297: Create and Alter Parquet tables and timezones; explicitTz = $explicitTz; " + + s"sessionTzOpt = $sessionTzOpt") { + val localSession = spark.newSession() + sessionTzOpt.foreach { tz => localSession.conf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, tz) } + val key = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY + withTable(baseTable, s"like_$baseTable", s"select_$baseTable", s"partitioned_$baseTable") { + // If we ever add a property to set the table timezone by default, defaultTz would change + val defaultTz = None + // check that created tables have correct TBLPROPERTIES + val tblProperties = explicitTz.map { + tz => s"""TBLPROPERTIES ($key="$tz")""" + }.getOrElse("") + localSession.sql( + s"""CREATE TABLE $baseTable ( + | x int + | ) + | STORED AS PARQUET + | $tblProperties + """.stripMargin) + val expectedTableTz = explicitTz.orElse(defaultTz) + checkHasTz(localSession, baseTable, expectedTableTz) + localSession.sql( + s"""CREATE TABLE partitioned_$baseTable ( + | x int + | ) + | PARTITIONED BY (y int) + | STORED AS PARQUET + | $tblProperties + """.stripMargin) + checkHasTz(localSession, s"partitioned_$baseTable", expectedTableTz) + localSession.sql(s"CREATE TABLE like_$baseTable LIKE $baseTable") + checkHasTz(localSession, s"like_$baseTable", expectedTableTz) + localSession.sql( + s"""CREATE TABLE select_$baseTable + | STORED AS PARQUET + | AS + | SELECT * from $baseTable + """.stripMargin) + checkHasTz(localSession, s"select_$baseTable", defaultTz) + + // check alter table, setting, unsetting, resetting the property + localSession.sql( + s"""ALTER TABLE $baseTable SET TBLPROPERTIES ($key="America/Los_Angeles")""") + checkHasTz(localSession, baseTable, Some("America/Los_Angeles")) + localSession.sql(s"""ALTER TABLE $baseTable SET TBLPROPERTIES ($key="UTC")""") + checkHasTz(localSession, baseTable, Some("UTC")) + localSession.sql(s"""ALTER TABLE $baseTable UNSET TBLPROPERTIES ($key)""") + checkHasTz(localSession, baseTable, None) + explicitTz.foreach { tz => + localSession.sql(s"""ALTER TABLE $baseTable SET TBLPROPERTIES ($key="$tz")""") + checkHasTz(localSession, baseTable, expectedTableTz) + } + } + } + } + + val desiredTimestampStrings = Seq( + "2015-12-31 22:49:59.123", + "2015-12-31 23:50:59.123", + "2016-01-01 00:39:59.123", + "2016-01-01 01:29:59.123" + ) + // We don't want to mess with timezones inside the tests themselves, since we use a shared + // spark context, and then we might be prone to issues from lazy vals for timezones. Instead, + // we manually adjust the timezone just to determine what the desired millis (since epoch, in utc) + // is for various "wall-clock" times in different timezones, and then we can compare against those + // in our tests. + val timestampTimezoneToMillis = { + val originalTz = TimeZone.getDefault + try { + desiredTimestampStrings.flatMap { timestampString => + Seq("America/Los_Angeles", "Europe/Berlin", "UTC").map { tzId => + TimeZone.setDefault(TimeZone.getTimeZone(tzId)) + val timestamp = Timestamp.valueOf(timestampString) + (timestampString, tzId) -> timestamp.getTime() + } + }.toMap + } finally { + TimeZone.setDefault(originalTz) + } + } + + private def createRawData(spark: SparkSession): Dataset[(String, Timestamp)] = { + import spark.implicits._ + val df = desiredTimestampStrings.toDF("display") + // this will get the millis corresponding to the display time given the current *session* + // timezone. + df.withColumn("ts", expr("cast(display as timestamp)")).as[(String, Timestamp)] + } + + test("SPARK-12297: Read and write with timezone adjustments") { + assert(TimeZone.getDefault.getID() === "America/Los_Angeles") + val key = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY + val originalData = createRawData(spark) + withTempPath { basePath => + val dsPath = new File(basePath, "dsFlat").getAbsolutePath + originalData.write + .option(key, "Europe/Berlin") + .parquet(dsPath) + + def checkRawData(data: DataFrame, tz: String): Unit = { + data.collect().foreach { row => + val disp = row.getAs[String]("display") + val ts = row.getAs[Timestamp]("ts") + assert(disp != ts.toString) + val expMillis = timestampTimezoneToMillis((disp, tz)) + assert(ts.getTime() === expMillis) + } + } + + // read it back, without supplying the right timezone. Won't match the original, but we + // expect specific values. + val readNoCorrection = spark.read.parquet(dsPath) + checkRawData(readNoCorrection, "Europe/Berlin") + + // now read it back *with* the right timezone -- everything should match. + val readWithCorrection = spark.read.option(key, "Europe/Berlin").parquet(dsPath) + + readWithCorrection.collect().foreach { row => + assert(row.getAs[String]("display") === row.getAs[Timestamp]("ts").toString()) + } + // now make sure it works if we read and write together + withTable("adjusted_table", "adjusted_table_2", "save_as_table") { + Seq("adjusted_table", "adjusted_table_2").foreach { table => + spark.sql(s"""CREATE EXTERNAL TABLE $table ( + | display string, + | ts timestamp + |) + |STORED AS parquet + |LOCATION 'file:${basePath.getAbsolutePath}/$table' + |TBLPROPERTIES ($key="UTC") + |""".stripMargin) + checkHasTz(spark, "adjusted_table", Some("UTC")) + } + + // do the read-write twice -- we want to make sure it works when the read-portion of the + // plan has already been analyzed, and also when it hasn't + Seq(false, true).foreach { fromHiveTable => + withClue(s"reading from ${if (fromHiveTable) "hive table" else "ds table"}") { + val dest = if (fromHiveTable) { + spark.sql("insert into adjusted_table_2 select * from adjusted_table") + // we dont' need to test CTAS, since CTAS tables won't ever have the table properties + "adjusted_table_2" + } else { + readWithCorrection.write.insertInto("adjusted_table") + "adjusted_table" + } + + val readFromTable = spark.sql(s"select display, ts from $dest") + val tableMeta = + spark.sessionState.catalog.getTableMetadata(TableIdentifier(dest)) + readFromTable.collect().foreach { row => + assert(row.getAs[String]("display") === row.getAs[Timestamp]("ts").toString()) + } + val tablePath = s"${basePath.getAbsolutePath}/$dest" + val readFromHiveUncorrected = spark.read.parquet(tablePath) + checkRawData(readFromHiveUncorrected, "UTC") + } + } + } + + readWithCorrection.write.option(key, "UTC").saveAsTable("save_as_table") + // I don't really understand why, but the timezone gets put into a table storage property, + // not a table property, this way. So round-tripping via spark works, but it won't + // be accessible to other engines. + checkHasTz(spark, "save_as_table", Some("UTC")) + + val readFromDsTable = spark.sql("select * from save_as_table") + readFromDsTable.collect().foreach { row => + assert(row.getAs[String]("display") === row.getAs[Timestamp]("ts").toString()) + } + val tableMeta = spark.sessionState.catalog.getTableMetadata(TableIdentifier("save_as_table")) + val readFromDsTableUncorrected = spark.read.parquet(tableMeta.location.toString) + checkRawData(readFromDsTableUncorrected, "UTC") + } + } + + test("SPARK-12297: exception on bad timezone") { + val key = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY + val badVal = "Blart Versenwald III" + def hasBadTzException(command: => Unit): Unit = { + withTable("bad_tz_table") { + val badTzException = intercept[AnalysisException] { command } + assert(badTzException.getMessage.contains(badVal)) + logWarning(badTzException.getMessage) + } + } + hasBadTzException{ + spark.sql( + s"""CREATE TABLE bad_tz_table ( + | x int + | ) + | STORED AS PARQUET + | TBLPROPERTIES ($key="$badVal") + """.stripMargin) + } + + spark.sql("CREATE TABLE bad_tz_table (x int)") + hasBadTzException { + spark.sql(s"""ALTER TABLE bad_tz_table SET TBLPROPERTIES($key="$badVal")""") + } + + hasBadTzException { + createRawData(spark).write.format("parquet").option(key, badVal).saveAsTable("blah") + } + + withTempPath { p => + hasBadTzException { + createRawData(spark).write.option(key, badVal).parquet(p.getAbsolutePath) + } + } + } + + test("SPARK-12297: insertInto must not specify timezone") { + // after you've already created a table, its too late to specify the timezone, so fail loudly + // if the user tries. + withTable("some_table") { + spark.sql("CREATE TABLE some_table (x string, y timestamp)") + val exc = intercept[AnalysisException]{ + createRawData(spark).write.option(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY, "UTC") + .insertInto("some_table") + } + assert(exc.getMessage.contains("Cannot provide a table timezone on insert")) + } + + } } From 53b9fbe0c6128ec11afdb46d3239c693129f6952 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 14 Sep 2017 15:18:46 -0500 Subject: [PATCH 02/23] All dataformats support timezone correction. Move rules & tests to a more appropriate location. Ensure rule works without hive support. Extra checks on when table timezones are set. --- .../expressions/datetimeExpressions.scala | 4 +- .../expressions/DateExpressionsSuite.scala | 4 +- .../apache/spark/sql/DataFrameReader.scala | 3 +- .../apache/spark/sql/DataFrameWriter.scala | 14 +- .../spark/sql/execution/command/ddl.scala | 12 +- .../spark/sql/execution/command/tables.scala | 4 +- .../spark/sql/execution/command/views.scala | 5 + .../TimestampTableTimeZone.scala} | 80 ++-- .../parquet/ParquetFileFormat.scala | 21 -- .../internal/BaseSessionStateBuilder.scala | 1 + .../TimestampTableTimeZoneSuite.scala | 355 ++++++++++++++++++ .../sql/hive/HiveSessionStateBuilder.scala | 4 +- .../CreateHiveTableAsSelectCommand.scala | 2 +- .../HiveTimestampTableTimeZoneSuite.scala | 87 +++++ .../hive/ParquetHiveCompatibilitySuite.scala | 261 +------------ 15 files changed, 515 insertions(+), 342 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/execution/{ParquetTimeZoneCorrection.scala => datasources/TimestampTableTimeZone.scala} (80%) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZoneSuite.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveTimestampTableTimeZoneSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index cca7f5636a399..75aa21bc6aafd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -1219,7 +1219,7 @@ case class ToUTCTimestamp(left: Expression, right: Expression) * We intentionally do not provide an ExpressionDescription as this is not meant to be exposed to * users, its only used for internal conversions. */ -private[spark] case class ParquetTimestampCorrection( +private[spark] case class TimestampTimezoneCorrection( time: Expression, from: Expression, to: Expression) @@ -1235,7 +1235,7 @@ private[spark] case class ParquetTimestampCorrection( override def children: Seq[Expression] = Seq(time, from, to) override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType, StringType, StringType) override def dataType: DataType = TimestampType - override def prettyName: String = "parquet_timestamp_correction" + override def prettyName: String = "timestamp_timezone_correction" override def nullSafeEval(time: Any, from: Any, to: Any): Any = { DateTimeUtils.convertTz( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index 2538da6eed4ea..a23926c94dffc 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -745,13 +745,13 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("parquet_timestamp_correction") { def test(t: String, fromTz: String, toTz: String, expected: String): Unit = { checkEvaluation( - ParquetTimestampCorrection( + TimestampTimezoneCorrection( Literal.create(if (t != null) Timestamp.valueOf(t) else null, TimestampType), Literal.create(fromTz, StringType), Literal.create(toTz, StringType)), if (expected != null) Timestamp.valueOf(expected) else null) checkEvaluation( - ParquetTimestampCorrection( + TimestampTimezoneCorrection( Literal.create(if (t != null) Timestamp.valueOf(t) else null, TimestampType), NonFoldableLiteral.create(fromTz, StringType), NonFoldableLiteral.create(toTz, StringType)), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index c69acc413e87f..019208a3a3988 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -28,7 +28,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.json.{CreateJacksonParser, JacksonParser, JSONOptions} import org.apache.spark.sql.execution.command.DDLUtils -import org.apache.spark.sql.execution.datasources.{DataSource, FailureSafeParser} +import org.apache.spark.sql.execution.datasources.{DataSource, FailureSafeParser, TimestampTableTimeZone} import org.apache.spark.sql.execution.datasources.csv._ import org.apache.spark.sql.execution.datasources.jdbc._ import org.apache.spark.sql.execution.datasources.json.TextInputJsonDataSource @@ -179,6 +179,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { throw new AnalysisException("Hive data source can only be used with tables, you can not " + "read files of Hive data source directly.") } + TimestampTableTimeZone.checkTableTz("", extraOptions.toMap) sparkSession.baseRelationToDataFrame( DataSource.apply( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 571cc683dab58..760d28e5d92d5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -28,8 +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.parquet.ParquetFileFormat +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 @@ -216,8 +215,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { */ def save(path: String): Unit = { this.extraOptions += ("path" -> path) - val destStr = s"for path $path" - ParquetFileFormat.checkTableTz(destStr, extraOptions.toMap) + TimestampTableTimeZone.checkTableTz(s"for path $path", extraOptions.toMap) save() } @@ -269,9 +267,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * @since 1.4.0 */ def insertInto(tableName: String): Unit = { - extraOptions.get(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY).foreach { tz => + extraOptions.get(TimestampTableTimeZone.TIMEZONE_PROPERTY).foreach { tz => throw new AnalysisException("Cannot provide a table timezone on insert; tried to insert " + - s"$tableName with ${ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY}=$tz") + s"$tableName with ${TimestampTableTimeZone.TIMEZONE_PROPERTY}=$tz") } insertInto(df.sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName)) } @@ -414,8 +412,8 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { CatalogTableType.MANAGED } val props = - extraOptions.filterKeys(key => key == ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY).toMap - ParquetFileFormat.checkTableTz(tableIdent, props) + extraOptions.filterKeys(key => key == TimestampTableTimeZone.TIMEZONE_PROPERTY).toMap + TimestampTableTimeZone.checkTableTz(tableIdent, props) val tableDesc = CatalogTable( identifier = tableIdent, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index e2331c0b03ced..6625f04f0ba1d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -34,10 +34,9 @@ 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.catalyst.util.DateTimeUtils -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.{ParquetFileFormat, ParquetSchemaConverter} +import org.apache.spark.sql.execution.datasources.parquet.ParquetSchemaConverter import org.apache.spark.sql.internal.HiveSerDe import org.apache.spark.sql.types._ import org.apache.spark.util.{SerializableConfiguration, ThreadUtils} @@ -231,7 +230,12 @@ case class AlterTableSetPropertiesCommand( isView: Boolean) extends RunnableCommand { - ParquetFileFormat.checkTableTz(tableName, properties) + 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 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index ab367e7c49d0a..59f3d69679257 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTableType._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.util.{quoteIdentifier, DateTimeUtils} -import org.apache.spark.sql.execution.datasources.{DataSource, PartitioningUtils} +import org.apache.spark.sql.execution.datasources.{DataSource, PartitioningUtils, TimestampTableTimeZone} import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat import org.apache.spark.sql.execution.datasources.json.JsonFileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat @@ -128,7 +128,7 @@ case class CreateTableCommand( Seq.empty[Row] } - ParquetFileFormat.checkTableTz(table.identifier, table.properties) + TimestampTableTimeZone.checkTableTz(table.identifier, table.properties) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index ffdfd527fa701..593dae1a93925 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable import org.apache.spark.sql.catalyst.expressions.{Alias, SubqueryExpression} import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, View} +import org.apache.spark.sql.execution.datasources.TimestampTableTimeZone import org.apache.spark.sql.types.MetadataBuilder import org.apache.spark.sql.util.SchemaUtils @@ -123,6 +124,10 @@ case class CreateViewCommand( s"It is not allowed to add database prefix `$database` for the TEMPORARY view name.") } + properties.get(TimestampTableTimeZone.TIMEZONE_PROPERTY).foreach { _ => + throw new AnalysisException("Timezone cannot be set for view") + } + override def run(sparkSession: SparkSession): Seq[Row] = { // If the plan cannot be analyzed, throw an exception and don't proceed. val qe = sparkSession.sessionState.executePlan(child) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ParquetTimeZoneCorrection.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZone.scala similarity index 80% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/ParquetTimeZoneCorrection.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZone.scala index 07a354d6c2065..9c080f7d7481c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ParquetTimeZoneCorrection.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZone.scala @@ -14,16 +14,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.execution +package org.apache.spark.sql.execution.datasources import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.command.CreateDataSourceTableAsSelectCommand -import org.apache.spark.sql.execution.datasources._ -import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types.{StringType, TimestampType} /** @@ -31,12 +30,13 @@ import org.apache.spark.sql.types.{StringType, TimestampType} * like TIMESTAMP WITHOUT TIMEZONE. This gives correct behavior if you process data with * machines in different timezones, or if you access the data from multiple SQL engines. */ -case class ParquetTimeZoneCorrection(sparkSession: SparkSession) extends Rule[LogicalPlan] { +private[sql] case class TimestampTableTimeZone(sparkSession: SparkSession) + extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = { // we can't use transformUp because we want to terminate recursion if there was already // timestamp correction, to keep this idempotent. - val r = plan match { + plan match { case insertIntoHadoopFs: InsertIntoHadoopFsRelationCommand => if (!insertIntoHadoopFs.query.isInstanceOf[Project]) { throw new AnalysisException("Expected the child of InsertIntoHadoopFsRelationCommand " + @@ -45,25 +45,13 @@ case class ParquetTimeZoneCorrection(sparkSession: SparkSession) extends Rule[Lo // The query might be reading from a parquet table which requires a different conversion; // this makes sure we apply the correct conversions there. val (fixedQuery, _) = readConversion(insertIntoHadoopFs.query) - - val tableTz = extractTableTz(insertIntoHadoopFs.catalogTable, insertIntoHadoopFs.options) - val x = tableTz.map{ tz => - writeConversion( - writer = insertIntoHadoopFs, - innerQuery = fixedQuery, - tableTz = tz, - copyWriter = insertIntoFsCopy(insertIntoHadoopFs, _) - ) - }.getOrElse(insertIntoHadoopFs) - logDebug(s"converted $plan to $x") - x + writeConversion(insertIntoHadoopFs.copy(query = fixedQuery)) case other => // recurse into children to see if we're reading data that needs conversion val (convertedPlan, _) = readConversion(plan) convertedPlan } - r } private def readConversion( @@ -117,7 +105,7 @@ case class ParquetTimeZoneCorrection(sparkSession: SparkSession) extends Rule[Lo var hasCorrection = false exprs.foreach { expr => expr.foreach { - case _: ParquetTimestampCorrection => + case _: TimestampTimezoneCorrection => hasCorrection = true case other => // no-op } @@ -131,36 +119,26 @@ case class ParquetTimeZoneCorrection(sparkSession: SparkSession) extends Rule[Lo insert.copy(query = newQuery) } - private def createDatasourceTableCopy( - insert: CreateDataSourceTableAsSelectCommand, - newQuery: LogicalPlan): CreateDataSourceTableAsSelectCommand = { - insert.copy(query = newQuery) - } - private def writeConversion( - writer: LogicalPlan, - innerQuery: LogicalPlan, - tableTz: String, - copyWriter: LogicalPlan => LogicalPlan): LogicalPlan = { + insertIntoHadoopFs: InsertIntoHadoopFsRelationCommand): InsertIntoHadoopFsRelationCommand = { + val query = insertIntoHadoopFs.query + val tableTz = extractTableTz(insertIntoHadoopFs.catalogTable, insertIntoHadoopFs.options) val internalTz = sparkSession.sessionState.conf.sessionLocalTimeZone - if (tableTz != internalTz) { + if (tableTz.isDefined && tableTz != internalTz) { val (foundTsFields, modifiedFields, _) = - convertTzForAllTimestamps(innerQuery, internalTz, tableTz) + convertTzForAllTimestamps(query, internalTz, tableTz.get) if (foundTsFields) { - val adjustedFields = - new Project(modifiedFields, innerQuery) - val converted = copyWriter(adjustedFields) - converted + insertIntoHadoopFs.copy(query = new Project(modifiedFields, query)) } else { - writer + insertIntoHadoopFs } } else { - writer + insertIntoHadoopFs } } private def extractTableTz(options: Map[String, String]): Option[String] = { - options.get(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY) + options.get(TimestampTableTimeZone.TIMEZONE_PROPERTY) } private def extractTableTz( @@ -192,7 +170,7 @@ case class ParquetTimeZoneCorrection(sparkSession: SparkSession) extends Rule[Lo if (field.dataType == TimestampType) { foundTs = true val adjustedTs = Alias( - ParquetTimestampCorrection( + TimestampTimezoneCorrection( exp, Literal.create(fromTz, StringType), Literal.create(toTz, StringType) @@ -210,3 +188,25 @@ case class ParquetTimeZoneCorrection(sparkSession: SparkSession) extends Rule[Lo (foundTs, modifiedFields, replacements) } } + +private[sql] object TimestampTableTimeZone { + val TIMEZONE_PROPERTY = "table.timezone" + /** + * Throw an AnalysisException if we're trying to set an invalid timezone for this table. + */ + private[sql] def checkTableTz(table: TableIdentifier, properties: Map[String, String]): Unit = { + checkTableTz(s"in table ${table.toString}", properties) + } + + /** + * Throw an AnalysisException if we're trying to set an invalid timezone for this table. + */ + private[sql] def checkTableTz(dest: String, properties: Map[String, String]): Unit = { + properties.get(TIMEZONE_PROPERTY).foreach { tz => + if (!DateTimeUtils.isValidTimezone(tz)) { + throw new AnalysisException(s"Cannot set $TIMEZONE_PROPERTY to invalid " + + s"timezone $tz $dest") + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index 64a124e883603..3340e0bb193b7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -646,25 +646,4 @@ object ParquetFileFormat extends Logging { Failure(cause) }.toOption } - - private[spark] val PARQUET_TIMEZONE_TABLE_PROPERTY = "parquet.mr.int96.write.zone" - - /** - * Throw an AnalysisException if we're trying to set an invalid timezone for this table. - */ - private[spark] def checkTableTz(table: TableIdentifier, properties: Map[String, String]): Unit = { - checkTableTz(s"in table ${table.toString}", properties) - } - - /** - * Throw an AnalysisException if we're trying to set an invalid timezone for this table. - */ - private[spark] def checkTableTz(dest: String, properties: Map[String, String]): Unit = { - properties.get(PARQUET_TIMEZONE_TABLE_PROPERTY).foreach { tz => - if (!DateTimeUtils.isValidTimezone(tz)) { - throw new AnalysisException(s"Cannot set $PARQUET_TIMEZONE_TABLE_PROPERTY to invalid " + - s"timezone $tz $dest") - } - } - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala index 4e756084bbdbb..efedbcf774b5f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala @@ -164,6 +164,7 @@ abstract class BaseSessionStateBuilder( PreprocessTableCreation(session) +: PreprocessTableInsertion(conf) +: DataSourceAnalysis(conf) +: + TimestampTableTimeZone(session) +: customPostHocResolutionRules override val extendedCheckRules: Seq[LogicalPlan => Unit] = diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZoneSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZoneSuite.scala new file mode 100644 index 0000000000000..be4a72c161fbe --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZoneSuite.scala @@ -0,0 +1,355 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources + +import java.io.File +import java.sql.Timestamp +import java.util.TimeZone + +import org.scalatest.BeforeAndAfterAll + +import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, SparkSession} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.execution.SparkPlanTest +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} + +abstract class BaseTimestampTableTimeZoneSuite extends SparkPlanTest with SQLTestUtils + with BeforeAndAfterAll { + + var originalTz: TimeZone = _ + protected override def beforeAll(): Unit = { + super.beforeAll() + originalTz = TimeZone.getDefault() + TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) + } + + protected override def afterAll(): Unit = { + TimeZone.setDefault(originalTz) + super.afterAll() + } + + val desiredTimestampStrings = Seq( + "2015-12-31 22:49:59.123", + "2015-12-31 23:50:59.123", + "2016-01-01 00:39:59.123", + "2016-01-01 01:29:59.123" + ) + // We don't want to mess with timezones inside the tests themselves, since we use a shared + // spark context, and then we might be prone to issues from lazy vals for timezones. Instead, + // we manually adjust the timezone just to determine what the desired millis (since epoch, in utc) + // is for various "wall-clock" times in different timezones, and then we can compare against those + // in our tests. + val timestampTimezoneToMillis = { + val originalTz = TimeZone.getDefault + try { + desiredTimestampStrings.flatMap { timestampString => + Seq("America/Los_Angeles", "Europe/Berlin", "UTC").map { tzId => + TimeZone.setDefault(TimeZone.getTimeZone(tzId)) + val timestamp = Timestamp.valueOf(timestampString) + (timestampString, tzId) -> timestamp.getTime() + } + }.toMap + } finally { + TimeZone.setDefault(originalTz) + } + } + + private def createRawData(spark: SparkSession): Dataset[(String, Timestamp)] = { + import spark.implicits._ + val df = desiredTimestampStrings.toDF("display") + // this will get the millis corresponding to the display time given the current *session* + // timezone. + df.withColumn("ts", expr("cast(display as timestamp)")).as[(String, Timestamp)] + } + + private def checkHasTz(spark: SparkSession, table: String, tz: Option[String]): Unit = { + val tableMetadata = spark.sessionState.catalog.getTableMetadata(TableIdentifier(table)) + assert(tableMetadata.properties.get(TimestampTableTimeZone.TIMEZONE_PROPERTY) === tz, + s"for table $table") + } + + /** + * This checks that the dataframe contains data that matches our original data set, modified for + * the given timezone, *if* we didn't apply any conversions when reading it back. You should + * pass in a dataframe that reads from the raw files, without any timezone specified. + */ + private def checkRawData(data: DataFrame, tz: String): Unit = { + val rows = data.collect() + assert(rows.size == 4) + rows.foreach { row => + val disp = row.getAs[String]("display") + val ts = row.getAs[Timestamp]("ts") + val expMillis = timestampTimezoneToMillis((disp, tz)) + assert(ts.getTime() === expMillis) + } + } + + // we want to test that this works w/ hive-only methods as well, so provide a few extension + // points so we can also easily re-use this with hive support. + protected def createAndSaveTableFunctions(): Seq[CreateAndSaveTable] = { + Seq(CreateAndSaveDatasourceTable) + } + protected def ctasFunctions(): Seq[CTAS] = { + Seq(DatasourceCTAS) + } + + trait CreateAndSaveTable { + /** + * if the format is unsupported return false (and do nothing else). + * otherwise, create the table, save the dataset into it, and return true + */ + def createAndSave( + ds: DataFrame, + table: String, + tz: Option[String], + format: String): Boolean + } + + object CreateAndSaveDatasourceTable extends CreateAndSaveTable { + override def createAndSave( + df: DataFrame, + table: String, + tz: Option[String], + format: String): Boolean = { + val writer = df.write.format(format) + tz.foreach { writer.option(TimestampTableTimeZone.TIMEZONE_PROPERTY, _)} + writer.saveAsTable(table) + true + } + } + + trait CTAS { + /** + * If the format is unsupported, return false (and do nothing else). Otherwise, create a table + * with the given time zone, and copy the entire contents of another table into it. + */ + def createTableFromSourceTable( + source: String, + dest: String, + destTz: Option[String], + destFormat: String): Boolean + } + + object DatasourceCTAS extends CTAS { + override def createTableFromSourceTable( + source: String, + dest: String, + destTz: Option[String], + destFormat: String): Boolean = { + val writer = spark.sql(s"select * from $source").write.format(destFormat) + destTz.foreach { writer.option(TimestampTableTimeZone.TIMEZONE_PROPERTY, _)} + writer.saveAsTable(dest) + true + } + } + + val formats = Seq("parquet", "csv", "json") + + test("SPARK-12297: Read and write with table timezones") { + assert(TimeZone.getDefault.getID() === "America/Los_Angeles") + val key = TimestampTableTimeZone.TIMEZONE_PROPERTY + val originalData = createRawData(spark) + withTempPath { basePath => + val dsPath = new File(basePath, "dsFlat").getAbsolutePath + originalData.write + .option(key, "Europe/Berlin") + .parquet(dsPath) + + /** + * Reads the raw data underlying the table, and assuming the data came from + * [[createRawData()]], make sure the values are correct. + */ + def checkTableData(table: String, format: String): Unit = { + // These queries should return the entire dataset, but if the predicates were + // applied to the raw values in parquet, they would incorrectly filter data out. + Seq( + ">" -> "2015-12-31 22:00:00", + "<" -> "2016-01-01 02:00:00" + ).foreach { case (comparison, value) => + val query = + s"select ts from $table where ts $comparison '$value'" + val countWithFilter = spark.sql(query).count() + assert(countWithFilter === 4, query) + } + + // also, read the raw parquet data, without any TZ correction, and make sure the raw + // values have been adjusted as we expect. + val tableMeta = spark.sessionState.catalog.getTableMetadata(TableIdentifier(table)) + val location = tableMeta.location.toString() + val tz = tableMeta.properties.get(TimestampTableTimeZone.TIMEZONE_PROPERTY) + // some formats need the schema specified + val df = spark.read.schema(originalData.schema).format(format).load(location) + checkRawData(df, tz.getOrElse("America/Los_Angeles")) + } + + // read it back, without supplying the right timezone. Won't match the original, but we + // expect specific values. + val readNoCorrection = spark.read.parquet(dsPath) + checkRawData(readNoCorrection, "Europe/Berlin") + + // now read it back *with* the right timezone -- everything should match. + val readWithCorrection = spark.read.option(key, "Europe/Berlin").parquet(dsPath) + + readWithCorrection.collect().foreach { row => + assert(row.getAs[String]("display") === row.getAs[Timestamp]("ts").toString()) + } + + // save to tables, and read the data back -- this time, the timezone conversion should be + // automatic from the table metadata, we don't need to supply any options when reading the + // data. Works across different ways of creating the tables and different data formats. + createAndSaveTableFunctions().foreach { createAndSave => + formats.foreach { format => + val tblName = s"save_$format" + withTable(tblName) { + // create the table (if we can -- not all createAndSave() methods support all formats, + // eg. hive tables don't support json) + if (createAndSave.createAndSave(readWithCorrection, tblName, Some("UTC"), format)) { + // make sure it has the right timezone, and the data is correct. + checkHasTz(spark, tblName, Some("UTC")) + checkTableData(tblName, format) + + // also try to copy this table directly into another table with a different timezone + // setting, for all formats. + ctasFunctions().foreach { ctas => + formats.foreach { destFormat => + val destTableUTC = s"copy_to_utc_$format" + val destTableNoTZ = s"copy_to_no_tz_$format" + withTable(destTableUTC, destTableNoTZ) { + val ctasSupported = ctas.createTableFromSourceTable(source = tblName, + dest = destTableUTC, destTz = Some("UTC"), destFormat = destFormat) + if (ctasSupported) { + checkHasTz(spark, destTableUTC, Some("UTC")) + checkTableData(destTableUTC, destFormat) + + ctas.createTableFromSourceTable(source = tblName, dest = destTableNoTZ, + destTz = None, destFormat = destFormat) + checkHasTz(spark, destTableNoTZ, None) + checkTableData(destTableNoTZ, destFormat) + + // By now, we've checked that the data in both tables is different in terms + // of the raw values on disk, but they are the same after we apply the + // timezone conversions from the table properties. Just to be extra-sure, + // we join the tables and make sure its OK. + val joinedRows = spark.sql( + s"""SELECT a.display, a.ts + |FROM $tblName AS a + |JOIN $destTableUTC AS b + |ON (a.ts = b.ts)""".stripMargin).collect() + assert(joinedRows.size === 4) + joinedRows.foreach { row => + assert(row.getAs[String]("display") === + row.getAs[Timestamp]("ts").toString()) + } + } + } + } + } + + // Finally, try changing the tbl timezone. This destroys integrity + // of the existing data, but at this point we're just checking we can change + // the metadata + spark.sql(s"""ALTER TABLE $tblName SET TBLPROPERTIES ($key="America/Los_Angeles")""") + checkHasTz(spark, tblName, Some("America/Los_Angeles")) + + spark.sql(s"""ALTER TABLE $tblName UNSET TBLPROPERTIES ($key)""") + checkHasTz(spark, tblName, None) + + spark.sql(s"""ALTER TABLE $tblName SET TBLPROPERTIES ($key="UTC")""") + checkHasTz(spark, tblName, Some("UTC")) + } + } + } + } + } + } + + test("SPARK-12297: exception on bad timezone") { + // make sure there is an exception anytime we try to read or write with a bad timezone + val key = TimestampTableTimeZone.TIMEZONE_PROPERTY + val badVal = "Blart Versenwald III" + val data = createRawData(spark) + def hasBadTzException(command: => Unit): Unit = { + withTable("bad_tz_table") { + val badTzException = intercept[AnalysisException] { command } + assert(badTzException.getMessage.contains(badVal)) + } + } + + withTempPath { p => + hasBadTzException { + data.write.option(key, badVal).parquet(p.getAbsolutePath) + } + + data.write.parquet(p.getAbsolutePath) + hasBadTzException { + spark.read.option(key, badVal).parquet(p.getAbsolutePath) + } + } + + createAndSaveTableFunctions().foreach { createAndSave => + hasBadTzException{ + createAndSave.createAndSave(data.toDF(), "bad_tz_table", Some(badVal), "parquet") + } + + createAndSave.createAndSave(data.toDF(), "bad_tz_table", None, "parquet") + hasBadTzException { + spark.sql(s"""ALTER TABLE bad_tz_table SET TBLPROPERTIES($key="$badVal")""") + } + } + } + + test("SPARK-12297: insertInto must not specify timezone") { + // You can't specify the timezone for just a portion of inserted data. You can only specify + // the timezone for the *entire* table (data previously in the table and any future data) so + // complain loudly if the user tries to set the timezone on an insert. + withTable("some_table") { + val origData = createRawData(spark) + origData.write.saveAsTable("some_table") + val exc = intercept[AnalysisException]{ + createRawData(spark).write.option(TimestampTableTimeZone.TIMEZONE_PROPERTY, "UTC") + .insertInto("some_table") + } + assert(exc.getMessage.contains("Cannot provide a table timezone on insert")) + } + } + + test("SPARK-12297: refuse table timezone on views") { + val key = TimestampTableTimeZone.TIMEZONE_PROPERTY + val originalData = createRawData(spark) + + withTable("ok_table") { + originalData.write.saveAsTable("ok_table") + withView("view_with_tz") { + val exc1 = intercept[AnalysisException]{ + spark.sql(s"""CREATE VIEW view_with_tz + | TBLPROPERTIES ($key="UTC") + | AS SELECT * FROM ok_table + """.stripMargin) + } + assert(exc1.getMessage.contains("Timezone cannot be set for view")) + spark.sql("CREATE VIEW view_with_tz AS SELECT * FROM ok_table") + val exc2 = intercept[AnalysisException]{ + spark.sql(s"""ALTER VIEW view_with_tz SET TBLPROPERTIES($key="UTC")""") + } + assert(exc2.getMessage.contains("Timezone cannot be set for view")) + } + } + } +} + +class TimestampTableTimeZoneSuite extends BaseTimestampTableTimeZoneSuite with SharedSQLContext diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala index 3df7faa16a825..987ec5da3376b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.Analyzer import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.{ParquetTimeZoneCorrection, SparkPlanner} +import org.apache.spark.sql.execution.SparkPlanner import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.hive.client.HiveClient import org.apache.spark.sql.internal.{BaseSessionStateBuilder, SessionResourceLoader, SessionState} @@ -80,7 +80,7 @@ class HiveSessionStateBuilder(session: SparkSession, parentState: Option[Session PreprocessTableInsertion(conf) +: DataSourceAnalysis(conf) +: HiveAnalysis +: - ParquetTimeZoneCorrection(session) +: + TimestampTableTimeZone(session) +: customPostHocResolutionRules override val extendedCheckRules: Seq[LogicalPlan => Unit] = diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala index 65e8b4e3c725c..55901a1fd5126 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala @@ -92,7 +92,7 @@ case class CreateHiveTableAsSelectCommand( } override def argString: String = { - s"[Database:${tableDesc.database}}, " + + s"[Database:${tableDesc.database}, " + s"TableName: ${tableDesc.identifier.table}, " + s"InsertIntoHiveTable]" } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveTimestampTableTimeZoneSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveTimestampTableTimeZoneSuite.scala new file mode 100644 index 0000000000000..7d680ad33ebfb --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveTimestampTableTimeZoneSuite.scala @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.hive + +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.execution.datasources.{BaseTimestampTableTimeZoneSuite, TimestampTableTimeZone} +import org.apache.spark.sql.hive.test.TestHiveSingleton + +class HiveTimestampTableTimeZoneSuite extends BaseTimestampTableTimeZoneSuite + with TestHiveSingleton { + + override protected def createAndSaveTableFunctions(): Seq[CreateAndSaveTable] = { + super.createAndSaveTableFunctions() ++ Seq(CreateHiveTableAndInsert) + } + + override protected def ctasFunctions(): Seq[CTAS] = { + super.ctasFunctions() ++ Seq(CreateHiveTableWithTimezoneAndInsert) + } + + object CreateHiveTableAndInsert extends CreateAndSaveTable { + override def createAndSave( + df: DataFrame, + table: String, + tzOpt: Option[String], + format: String): Boolean = { + if (format == "parquet") { + val tblProperties = tzOpt.map { tz => + s"""TBLPROPERTIES (${TimestampTableTimeZone.TIMEZONE_PROPERTY}="$tz")""" + }.getOrElse("") + spark.sql( + s"""CREATE TABLE $table ( + | display string, + | ts timestamp + |) + |STORED AS parquet + |$tblProperties + |""".stripMargin) + df.write.insertInto(table) + true + } else { + false + } + } + } + + object CreateHiveTableWithTimezoneAndInsert extends CTAS { + override def createTableFromSourceTable( + source: String, + dest: String, + destTz: Option[String], + destFormat: String): Boolean = { + if (destFormat == "parquet") { + val tblProperties = destTz.map { tz => + s"""TBLPROPERTIES (${TimestampTableTimeZone.TIMEZONE_PROPERTY}="$tz")""" + }.getOrElse("") + // this isn't just a "ctas" sql statement b/c that doesn't let us specify the table tz + spark.sql( + s"""CREATE TABLE $dest ( + | display string, + | ts timestamp + |) + |STORED AS parquet + |$tblProperties + |""".stripMargin) + spark.sql(s"insert into $dest select * from $source") + true + } else { + false + } + + } + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala index 7b067845bcbae..05b6059472f59 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala @@ -17,22 +17,12 @@ package org.apache.spark.sql.hive -import java.io.File -import java.net.URLDecoder import java.sql.Timestamp -import java.util.TimeZone -import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.parquet.hadoop.ParquetFileReader -import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName - -import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.execution.datasources.parquet.{ParquetCompatibilityTest, ParquetFileFormat} -import org.apache.spark.sql.functions._ +import org.apache.spark.sql.Row +import org.apache.spark.sql.execution.datasources.parquet.ParquetCompatibilityTest import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{StringType, StructType, TimestampType} class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHiveSingleton { /** @@ -151,251 +141,4 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi Row(Seq(Row(1))), "ARRAY>") } - - // Check creating parquet tables with timestamps, writing data into them, and reading it back out - // under a variety of conditions: - // * tables with explicit tz and those without - // * altering table properties directly - // * variety of timezones, local & non-local - testCreateAlterTablesWithTimezone("no_tz", None, None) - testCreateAlterTablesWithTimezone("LA", Some("America/Los_Angeles"), None) - testCreateAlterTablesWithTimezone("LA", Some("Europe/Berlin"), None) - testCreateAlterTablesWithTimezone("LA", Some("Europe/Berlin"), Some("UTC")) - - private def checkHasTz(spark: SparkSession, table: String, tz: Option[String]): Unit = { - val tableMetadata = spark.sessionState.catalog.getTableMetadata(TableIdentifier(table)) - assert(tableMetadata.properties.get(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY) === tz, - s"for table $table") - } - - private def testCreateAlterTablesWithTimezone( - baseTable: String, - explicitTz: Option[String], - sessionTzOpt: Option[String]): Unit = { - test(s"SPARK-12297: Create and Alter Parquet tables and timezones; explicitTz = $explicitTz; " + - s"sessionTzOpt = $sessionTzOpt") { - val localSession = spark.newSession() - sessionTzOpt.foreach { tz => localSession.conf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, tz) } - val key = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY - withTable(baseTable, s"like_$baseTable", s"select_$baseTable", s"partitioned_$baseTable") { - // If we ever add a property to set the table timezone by default, defaultTz would change - val defaultTz = None - // check that created tables have correct TBLPROPERTIES - val tblProperties = explicitTz.map { - tz => s"""TBLPROPERTIES ($key="$tz")""" - }.getOrElse("") - localSession.sql( - s"""CREATE TABLE $baseTable ( - | x int - | ) - | STORED AS PARQUET - | $tblProperties - """.stripMargin) - val expectedTableTz = explicitTz.orElse(defaultTz) - checkHasTz(localSession, baseTable, expectedTableTz) - localSession.sql( - s"""CREATE TABLE partitioned_$baseTable ( - | x int - | ) - | PARTITIONED BY (y int) - | STORED AS PARQUET - | $tblProperties - """.stripMargin) - checkHasTz(localSession, s"partitioned_$baseTable", expectedTableTz) - localSession.sql(s"CREATE TABLE like_$baseTable LIKE $baseTable") - checkHasTz(localSession, s"like_$baseTable", expectedTableTz) - localSession.sql( - s"""CREATE TABLE select_$baseTable - | STORED AS PARQUET - | AS - | SELECT * from $baseTable - """.stripMargin) - checkHasTz(localSession, s"select_$baseTable", defaultTz) - - // check alter table, setting, unsetting, resetting the property - localSession.sql( - s"""ALTER TABLE $baseTable SET TBLPROPERTIES ($key="America/Los_Angeles")""") - checkHasTz(localSession, baseTable, Some("America/Los_Angeles")) - localSession.sql(s"""ALTER TABLE $baseTable SET TBLPROPERTIES ($key="UTC")""") - checkHasTz(localSession, baseTable, Some("UTC")) - localSession.sql(s"""ALTER TABLE $baseTable UNSET TBLPROPERTIES ($key)""") - checkHasTz(localSession, baseTable, None) - explicitTz.foreach { tz => - localSession.sql(s"""ALTER TABLE $baseTable SET TBLPROPERTIES ($key="$tz")""") - checkHasTz(localSession, baseTable, expectedTableTz) - } - } - } - } - - val desiredTimestampStrings = Seq( - "2015-12-31 22:49:59.123", - "2015-12-31 23:50:59.123", - "2016-01-01 00:39:59.123", - "2016-01-01 01:29:59.123" - ) - // We don't want to mess with timezones inside the tests themselves, since we use a shared - // spark context, and then we might be prone to issues from lazy vals for timezones. Instead, - // we manually adjust the timezone just to determine what the desired millis (since epoch, in utc) - // is for various "wall-clock" times in different timezones, and then we can compare against those - // in our tests. - val timestampTimezoneToMillis = { - val originalTz = TimeZone.getDefault - try { - desiredTimestampStrings.flatMap { timestampString => - Seq("America/Los_Angeles", "Europe/Berlin", "UTC").map { tzId => - TimeZone.setDefault(TimeZone.getTimeZone(tzId)) - val timestamp = Timestamp.valueOf(timestampString) - (timestampString, tzId) -> timestamp.getTime() - } - }.toMap - } finally { - TimeZone.setDefault(originalTz) - } - } - - private def createRawData(spark: SparkSession): Dataset[(String, Timestamp)] = { - import spark.implicits._ - val df = desiredTimestampStrings.toDF("display") - // this will get the millis corresponding to the display time given the current *session* - // timezone. - df.withColumn("ts", expr("cast(display as timestamp)")).as[(String, Timestamp)] - } - - test("SPARK-12297: Read and write with timezone adjustments") { - assert(TimeZone.getDefault.getID() === "America/Los_Angeles") - val key = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY - val originalData = createRawData(spark) - withTempPath { basePath => - val dsPath = new File(basePath, "dsFlat").getAbsolutePath - originalData.write - .option(key, "Europe/Berlin") - .parquet(dsPath) - - def checkRawData(data: DataFrame, tz: String): Unit = { - data.collect().foreach { row => - val disp = row.getAs[String]("display") - val ts = row.getAs[Timestamp]("ts") - assert(disp != ts.toString) - val expMillis = timestampTimezoneToMillis((disp, tz)) - assert(ts.getTime() === expMillis) - } - } - - // read it back, without supplying the right timezone. Won't match the original, but we - // expect specific values. - val readNoCorrection = spark.read.parquet(dsPath) - checkRawData(readNoCorrection, "Europe/Berlin") - - // now read it back *with* the right timezone -- everything should match. - val readWithCorrection = spark.read.option(key, "Europe/Berlin").parquet(dsPath) - - readWithCorrection.collect().foreach { row => - assert(row.getAs[String]("display") === row.getAs[Timestamp]("ts").toString()) - } - // now make sure it works if we read and write together - withTable("adjusted_table", "adjusted_table_2", "save_as_table") { - Seq("adjusted_table", "adjusted_table_2").foreach { table => - spark.sql(s"""CREATE EXTERNAL TABLE $table ( - | display string, - | ts timestamp - |) - |STORED AS parquet - |LOCATION 'file:${basePath.getAbsolutePath}/$table' - |TBLPROPERTIES ($key="UTC") - |""".stripMargin) - checkHasTz(spark, "adjusted_table", Some("UTC")) - } - - // do the read-write twice -- we want to make sure it works when the read-portion of the - // plan has already been analyzed, and also when it hasn't - Seq(false, true).foreach { fromHiveTable => - withClue(s"reading from ${if (fromHiveTable) "hive table" else "ds table"}") { - val dest = if (fromHiveTable) { - spark.sql("insert into adjusted_table_2 select * from adjusted_table") - // we dont' need to test CTAS, since CTAS tables won't ever have the table properties - "adjusted_table_2" - } else { - readWithCorrection.write.insertInto("adjusted_table") - "adjusted_table" - } - - val readFromTable = spark.sql(s"select display, ts from $dest") - val tableMeta = - spark.sessionState.catalog.getTableMetadata(TableIdentifier(dest)) - readFromTable.collect().foreach { row => - assert(row.getAs[String]("display") === row.getAs[Timestamp]("ts").toString()) - } - val tablePath = s"${basePath.getAbsolutePath}/$dest" - val readFromHiveUncorrected = spark.read.parquet(tablePath) - checkRawData(readFromHiveUncorrected, "UTC") - } - } - } - - readWithCorrection.write.option(key, "UTC").saveAsTable("save_as_table") - // I don't really understand why, but the timezone gets put into a table storage property, - // not a table property, this way. So round-tripping via spark works, but it won't - // be accessible to other engines. - checkHasTz(spark, "save_as_table", Some("UTC")) - - val readFromDsTable = spark.sql("select * from save_as_table") - readFromDsTable.collect().foreach { row => - assert(row.getAs[String]("display") === row.getAs[Timestamp]("ts").toString()) - } - val tableMeta = spark.sessionState.catalog.getTableMetadata(TableIdentifier("save_as_table")) - val readFromDsTableUncorrected = spark.read.parquet(tableMeta.location.toString) - checkRawData(readFromDsTableUncorrected, "UTC") - } - } - - test("SPARK-12297: exception on bad timezone") { - val key = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY - val badVal = "Blart Versenwald III" - def hasBadTzException(command: => Unit): Unit = { - withTable("bad_tz_table") { - val badTzException = intercept[AnalysisException] { command } - assert(badTzException.getMessage.contains(badVal)) - logWarning(badTzException.getMessage) - } - } - hasBadTzException{ - spark.sql( - s"""CREATE TABLE bad_tz_table ( - | x int - | ) - | STORED AS PARQUET - | TBLPROPERTIES ($key="$badVal") - """.stripMargin) - } - - spark.sql("CREATE TABLE bad_tz_table (x int)") - hasBadTzException { - spark.sql(s"""ALTER TABLE bad_tz_table SET TBLPROPERTIES($key="$badVal")""") - } - - hasBadTzException { - createRawData(spark).write.format("parquet").option(key, badVal).saveAsTable("blah") - } - - withTempPath { p => - hasBadTzException { - createRawData(spark).write.option(key, badVal).parquet(p.getAbsolutePath) - } - } - } - - test("SPARK-12297: insertInto must not specify timezone") { - // after you've already created a table, its too late to specify the timezone, so fail loudly - // if the user tries. - withTable("some_table") { - spark.sql("CREATE TABLE some_table (x string, y timestamp)") - val exc = intercept[AnalysisException]{ - createRawData(spark).write.option(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY, "UTC") - .insertInto("some_table") - } - assert(exc.getMessage.contains("Cannot provide a table timezone on insert")) - } - - } } From 5105b728dafa821d6063af97cfad0f49f029726a Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 15 Sep 2017 14:41:06 -0500 Subject: [PATCH 03/23] revert unintended changes --- .../parquet/ParquetFileFormat.scala | 3 +-- sql/hive/src/test/resources/log4j.properties | 21 ++----------------- 2 files changed, 3 insertions(+), 21 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index 3340e0bb193b7..e1e740500205a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -42,11 +42,10 @@ import org.apache.parquet.schema.MessageType import org.apache.spark.{SparkException, TaskContext} import org.apache.spark.internal.Logging import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.catalyst.parser.LegacyTypeStringParser -import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector import org.apache.spark.sql.internal.SQLConf diff --git a/sql/hive/src/test/resources/log4j.properties b/sql/hive/src/test/resources/log4j.properties index c334d24a11b76..a48ae9fc5edd8 100644 --- a/sql/hive/src/test/resources/log4j.properties +++ b/sql/hive/src/test/resources/log4j.properties @@ -16,19 +16,13 @@ # # Set everything to be logged to the file core/target/unit-tests.log -log4j.rootLogger=WARN, CA, FA +log4j.rootLogger=DEBUG, CA, FA #Console Appender log4j.appender.CA=org.apache.log4j.ConsoleAppender log4j.appender.CA.layout=org.apache.log4j.PatternLayout log4j.appender.CA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %p %c: %m%n -#log4j.appender.CA.Threshold = WARN -#log4j.logger=WARN, CA -#log4j.logger.org=WARN, CA -#log4j.logger.com=WARN, CA -#log4j.logger.io=WARN, CA -#log4j.logger.sun=WARN, CA -#log4j.logger.akka=WARN, CA +log4j.appender.CA.Threshold = WARN #File Appender @@ -38,17 +32,6 @@ log4j.appender.FA.file=target/unit-tests.log log4j.appender.FA.layout=org.apache.log4j.PatternLayout log4j.appender.FA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %t %p %c{1}: %m%n - -# to get more logs, have to turn off Threshold line -# log4j.logger.org.apache.spark.sql.catalyst.expressions.codegen=DEBUG -# log4j.logger.org.apache.spark.sql.execution.WholeStageCodegenExec=DEBUG -# log4j.logger.org.apache.spark.sql.execution.ParquetTimeZoneCorrection=DEBUG -# analyzer -# log4j.logger.org.apache.spark.sql.hive.HiveSessionStateBuilder$$anon$1=TRACE -# optimizer -#log4j.logger.org.apache.spark.sql.internal.BaseSessionStateBuilder$$anon$2=TRACE - - # Set the logger level of File Appender to WARN log4j.appender.FA.Threshold = DEBUG From c5571a8de509954ed2e90e4954a2e77ac7ad9627 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 15 Sep 2017 16:23:54 -0500 Subject: [PATCH 04/23] remove child restriction --- .../sql/execution/datasources/TimestampTableTimeZone.scala | 4 ---- 1 file changed, 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZone.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZone.scala index 9c080f7d7481c..aad36ebb97c14 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZone.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZone.scala @@ -38,10 +38,6 @@ private[sql] case class TimestampTableTimeZone(sparkSession: SparkSession) // timestamp correction, to keep this idempotent. plan match { case insertIntoHadoopFs: InsertIntoHadoopFsRelationCommand => - if (!insertIntoHadoopFs.query.isInstanceOf[Project]) { - throw new AnalysisException("Expected the child of InsertIntoHadoopFsRelationCommand " + - s"to be a Project in $plan") - } // The query might be reading from a parquet table which requires a different conversion; // this makes sure we apply the correct conversions there. val (fixedQuery, _) = readConversion(insertIntoHadoopFs.query) From 950d33a4835e56748963dfe002bfa7145d91469f Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 18 Sep 2017 11:02:58 -0500 Subject: [PATCH 05/23] fixes from tests --- .../org/apache/spark/sql/execution/command/tables.scala | 3 +-- .../execution/datasources/TimestampTableTimeZone.scala | 9 ++++++++- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 59f3d69679257..4b5f0068c04f5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -86,8 +86,7 @@ case class CreateTableLikeCommand( schema = sourceTableDesc.schema, provider = newProvider, partitionColumnNames = sourceTableDesc.partitionColumnNames, - bucketSpec = sourceTableDesc.bucketSpec, - properties = sourceTableDesc.properties) + bucketSpec = sourceTableDesc.bucketSpec) catalog.createTable(newTableDesc, ifNotExists) Seq.empty[Row] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZone.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZone.scala index aad36ebb97c14..790e59b717a25 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZone.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZone.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.datasources import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.UnresolvedException import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} @@ -89,7 +90,13 @@ private[sql] case class TimestampTableTimeZone(sparkSession: SparkSession) // now we need to adjust all names to use the new version. val fixedExpressions = fixedPlan.mapExpressions { outerExp => val adjustedExp = outerExp.transformUp { case exp: NamedExpression => - newReplacements.get(exp.exprId).getOrElse(exp) + try { + newReplacements.get(exp.exprId).getOrElse(exp) + } catch { + // UnresolvedAttributes etc. will cause problems later anyway, we just dont' want to + // expose the error here + case ue: UnresolvedException[_] => exp + } } logDebug(s"adjusted $outerExp to $adjustedExp using $newReplacements") adjustedExp From e36851e49dddb64b7f11a73f4115363b95c83667 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 18 Sep 2017 13:56:26 -0500 Subject: [PATCH 06/23] fix --- .../catalyst/expressions/datetimeExpressions.scala | 2 -- .../catalyst/expressions/DateExpressionsSuite.scala | 12 +++++++++--- 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index 75aa21bc6aafd..463c7ffd4d911 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -1225,8 +1225,6 @@ private[spark] case class TimestampTimezoneCorrection( to: Expression) extends TernaryExpression with ImplicitCastInputTypes { - // modeled on ToUTCTimestamp + Conv (as an example TernaryExpression) - // convertTz() does the *opposite* conversion we want, which is why from & to appear reversed // in all the calls to convertTz. Its used for showing how the display time changes when we go // from one timezone to another. We want to see how should change the SQLTimestamp value to diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index a23926c94dffc..0cf448f718ff2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -742,7 +742,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test(null, null, null) } - test("parquet_timestamp_correction") { + test("timestamp_timezone_correction") { def test(t: String, fromTz: String, toTz: String, expected: String): Unit = { checkEvaluation( TimestampTimezoneCorrection( @@ -757,8 +757,14 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { NonFoldableLiteral.create(toTz, StringType)), if (expected != null) Timestamp.valueOf(expected) else null) } - test("2015-07-24 00:00:00", "UTC", "PST", "2015-07-23 17:00:00") - test("2015-01-24 00:00:00", "UTC", "PST", "2015-01-23 16:00:00") + // These conversions may look backwards -- but this is *NOT* saying: + // when the clock says 2015-07-24 00:00:00 in PST, what would it say to somebody in UTC? + // Instead, its saying -- suppose somebody stored "2015-07-24 00:00:00" while in PST, but + // as millis-since-epoch. What millis-since-epoch would I need to also see + // "2015-07-24 00:00:00" if my clock were in UTC? Just for testing convenience, we input + // that last value as "what would my clock in PST say for that final millis-since-epoch?" + test("2015-07-24 00:00:00", "PST", "UTC", "2015-07-23 17:00:00") + test("2015-01-24 00:00:00", "PST", "UTC", "2015-01-23 16:00:00") test(null, "UTC", "UTC", null) test("2015-07-24 00:00:00", null, null, null) test(null, null, null, null) From 2069b650245a075c4d43884b242a157dd0e1a4f0 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 19 Sep 2017 09:03:45 -0500 Subject: [PATCH 07/23] cleanup --- .../datasources/TimestampTableTimeZone.scala | 26 +++++++++---------- .../TimestampTableTimeZoneSuite.scala | 11 ++++---- 2 files changed, 17 insertions(+), 20 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZone.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZone.scala index 790e59b717a25..2ab6e210f2d59 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZone.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZone.scala @@ -41,29 +41,33 @@ private[sql] case class TimestampTableTimeZone(sparkSession: SparkSession) case insertIntoHadoopFs: InsertIntoHadoopFsRelationCommand => // The query might be reading from a parquet table which requires a different conversion; // this makes sure we apply the correct conversions there. - val (fixedQuery, _) = readConversion(insertIntoHadoopFs.query) + val (fixedQuery, _) = convertInputs(insertIntoHadoopFs.query) writeConversion(insertIntoHadoopFs.copy(query = fixedQuery)) case other => // recurse into children to see if we're reading data that needs conversion - val (convertedPlan, _) = readConversion(plan) + val (convertedPlan, _) = convertInputs(plan) convertedPlan } } - private def readConversion( + /** + * Apply the correction to all timestamp inputs, and replace all references to the raw attributes + * with the new converted inputs. + * @return The converted plan, and the replacements to be applied further up the plan + */ + private def convertInputs( plan: LogicalPlan ): (LogicalPlan, Map[ExprId, NamedExpression]) = plan match { case alreadyConverted@Project(exprs, _) if hasCorrection(exprs) => - logDebug(s"not transforming $alreadyConverted because its already converted") (alreadyConverted, Map()) case lr@LogicalRelation(fsRelation: HadoopFsRelation, _, _, _) => val tzOpt = extractTableTz(lr.catalogTable, fsRelation.options) tzOpt.map { tableTz => // the table has a timezone set, so after reading the data, apply a conversion - // SessionTZ will make the time display correctly in SQL queries, but incorrectly if you - // pull Timestamp objects out (eg. with a dataset.collect()) + // SessionTZ (instead of JVM TZ) will make the time display correctly in SQL queries, but + // incorrectly if you pull Timestamp objects out (eg. with a dataset.collect()) val toTz = sparkSession.sessionState.conf.sessionLocalTimeZone if (toTz != tableTz) { logDebug(s"table tz = $tableTz; converting to current session tz = $toTz") @@ -83,7 +87,7 @@ private[sql] case class TimestampTableTimeZone(sparkSession: SparkSession) // first, process all the children -- this ensures we have the right renames in scope. var newReplacements = Map[ExprId, NamedExpression]() val fixedPlan = other.mapChildren { originalPlan => - val (newPlan, extraReplacements) = readConversion(originalPlan) + val (newPlan, extraReplacements) = convertInputs(originalPlan) newReplacements ++= extraReplacements newPlan } @@ -98,7 +102,6 @@ private[sql] case class TimestampTableTimeZone(sparkSession: SparkSession) case ue: UnresolvedException[_] => exp } } - logDebug(s"adjusted $outerExp to $adjustedExp using $newReplacements") adjustedExp } (fixedExpressions, newReplacements) @@ -116,12 +119,6 @@ private[sql] case class TimestampTableTimeZone(sparkSession: SparkSession) hasCorrection } - private def insertIntoFsCopy( - insert: InsertIntoHadoopFsRelationCommand, - newQuery: LogicalPlan): InsertIntoHadoopFsRelationCommand = { - insert.copy(query = newQuery) - } - private def writeConversion( insertIntoHadoopFs: InsertIntoHadoopFsRelationCommand): InsertIntoHadoopFsRelationCommand = { val query = insertIntoHadoopFs.query @@ -194,6 +191,7 @@ private[sql] case class TimestampTableTimeZone(sparkSession: SparkSession) private[sql] object TimestampTableTimeZone { val TIMEZONE_PROPERTY = "table.timezone" + /** * Throw an AnalysisException if we're trying to set an invalid timezone for this table. */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZoneSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZoneSuite.scala index be4a72c161fbe..9b6b6b4b2bbdf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZoneSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZoneSuite.scala @@ -50,10 +50,10 @@ abstract class BaseTimestampTableTimeZoneSuite extends SparkPlanTest with SQLTes "2016-01-01 01:29:59.123" ) // We don't want to mess with timezones inside the tests themselves, since we use a shared - // spark context, and then we might be prone to issues from lazy vals for timezones. Instead, - // we manually adjust the timezone just to determine what the desired millis (since epoch, in utc) - // is for various "wall-clock" times in different timezones, and then we can compare against those - // in our tests. + // spark context in the hive tests, and then we might be prone to issues from lazy vals for + // timezones. Instead, we manually adjust the timezone just to determine what the desired millis + // (since epoch, in utc) is for various "wall-clock" times in different timezones, and then we can + // compare against those in our tests. val timestampTimezoneToMillis = { val originalTz = TimeZone.getDefault try { @@ -72,8 +72,7 @@ abstract class BaseTimestampTableTimeZoneSuite extends SparkPlanTest with SQLTes private def createRawData(spark: SparkSession): Dataset[(String, Timestamp)] = { import spark.implicits._ val df = desiredTimestampStrings.toDF("display") - // this will get the millis corresponding to the display time given the current *session* - // timezone. + // this will get the millis corresponding to the display time given the current session tz df.withColumn("ts", expr("cast(display as timestamp)")).as[(String, Timestamp)] } From 74a9905a4b50e4c357f579bf3e237834bac3551e Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 19 Sep 2017 09:12:19 -0500 Subject: [PATCH 08/23] rename property to table.timezone-adjustment --- .../sql/execution/datasources/TimestampTableTimeZone.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZone.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZone.scala index 2ab6e210f2d59..edf02415d5ce9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZone.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZone.scala @@ -190,7 +190,7 @@ private[sql] case class TimestampTableTimeZone(sparkSession: SparkSession) } private[sql] object TimestampTableTimeZone { - val TIMEZONE_PROPERTY = "table.timezone" + val TIMEZONE_PROPERTY = "table.timezone-adjustment" /** * Throw an AnalysisException if we're trying to set an invalid timezone for this table. From 515b38b7b0a1efc0de4a92479c1e5a479b872146 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 19 Sep 2017 13:10:49 -0500 Subject: [PATCH 09/23] quotes around property name --- .../datasources/TimestampTableTimeZoneSuite.scala | 13 +++++++------ .../sql/hive/HiveTimestampTableTimeZoneSuite.scala | 4 ++-- 2 files changed, 9 insertions(+), 8 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZoneSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZoneSuite.scala index 9b6b6b4b2bbdf..90fb9e53d9f80 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZoneSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZoneSuite.scala @@ -262,13 +262,14 @@ abstract class BaseTimestampTableTimeZoneSuite extends SparkPlanTest with SQLTes // Finally, try changing the tbl timezone. This destroys integrity // of the existing data, but at this point we're just checking we can change // the metadata - spark.sql(s"""ALTER TABLE $tblName SET TBLPROPERTIES ($key="America/Los_Angeles")""") + spark.sql( + s"""ALTER TABLE $tblName SET TBLPROPERTIES ("$key"="America/Los_Angeles")""") checkHasTz(spark, tblName, Some("America/Los_Angeles")) - spark.sql(s"""ALTER TABLE $tblName UNSET TBLPROPERTIES ($key)""") + spark.sql(s"""ALTER TABLE $tblName UNSET TBLPROPERTIES ("$key")""") checkHasTz(spark, tblName, None) - spark.sql(s"""ALTER TABLE $tblName SET TBLPROPERTIES ($key="UTC")""") + spark.sql(s"""ALTER TABLE $tblName SET TBLPROPERTIES ("$key"="UTC")""") checkHasTz(spark, tblName, Some("UTC")) } } @@ -307,7 +308,7 @@ abstract class BaseTimestampTableTimeZoneSuite extends SparkPlanTest with SQLTes createAndSave.createAndSave(data.toDF(), "bad_tz_table", None, "parquet") hasBadTzException { - spark.sql(s"""ALTER TABLE bad_tz_table SET TBLPROPERTIES($key="$badVal")""") + spark.sql(s"""ALTER TABLE bad_tz_table SET TBLPROPERTIES("$key"="$badVal")""") } } } @@ -336,14 +337,14 @@ abstract class BaseTimestampTableTimeZoneSuite extends SparkPlanTest with SQLTes withView("view_with_tz") { val exc1 = intercept[AnalysisException]{ spark.sql(s"""CREATE VIEW view_with_tz - | TBLPROPERTIES ($key="UTC") + | TBLPROPERTIES ("$key"="UTC") | AS SELECT * FROM ok_table """.stripMargin) } assert(exc1.getMessage.contains("Timezone cannot be set for view")) spark.sql("CREATE VIEW view_with_tz AS SELECT * FROM ok_table") val exc2 = intercept[AnalysisException]{ - spark.sql(s"""ALTER VIEW view_with_tz SET TBLPROPERTIES($key="UTC")""") + spark.sql(s"""ALTER VIEW view_with_tz SET TBLPROPERTIES("$key"="UTC")""") } assert(exc2.getMessage.contains("Timezone cannot be set for view")) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveTimestampTableTimeZoneSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveTimestampTableTimeZoneSuite.scala index 7d680ad33ebfb..f1cbd90d28acb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveTimestampTableTimeZoneSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveTimestampTableTimeZoneSuite.scala @@ -39,7 +39,7 @@ class HiveTimestampTableTimeZoneSuite extends BaseTimestampTableTimeZoneSuite format: String): Boolean = { if (format == "parquet") { val tblProperties = tzOpt.map { tz => - s"""TBLPROPERTIES (${TimestampTableTimeZone.TIMEZONE_PROPERTY}="$tz")""" + s"""TBLPROPERTIES ("${TimestampTableTimeZone.TIMEZONE_PROPERTY}"="$tz")""" }.getOrElse("") spark.sql( s"""CREATE TABLE $table ( @@ -65,7 +65,7 @@ class HiveTimestampTableTimeZoneSuite extends BaseTimestampTableTimeZoneSuite destFormat: String): Boolean = { if (destFormat == "parquet") { val tblProperties = destTz.map { tz => - s"""TBLPROPERTIES (${TimestampTableTimeZone.TIMEZONE_PROPERTY}="$tz")""" + s"""TBLPROPERTIES ("${TimestampTableTimeZone.TIMEZONE_PROPERTY}"="$tz")""" }.getOrElse("") // this isn't just a "ctas" sql statement b/c that doesn't let us specify the table tz spark.sql( From a869c6e27009f1653a15cb39ae2e320e541d2a3e Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 21 Sep 2017 10:59:16 -0500 Subject: [PATCH 10/23] CREATE TABLE LIKE copies *only* the table timezone --- .../spark/sql/execution/command/tables.scala | 5 ++- .../TimestampTableTimeZoneSuite.scala | 8 ++--- .../HiveTimestampTableTimeZoneSuite.scala | 34 +++++++++++++++---- 3 files changed, 36 insertions(+), 11 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 4b5f0068c04f5..ec3e90d64ccb8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -76,6 +76,8 @@ case class CreateTableLikeCommand( // If the location is specified, we create an external table internally. // Otherwise create a managed table. val tblType = if (location.isEmpty) CatalogTableType.MANAGED else CatalogTableType.EXTERNAL + val properties = + sourceTableDesc.properties.filterKeys(_ == TimestampTableTimeZone.TIMEZONE_PROPERTY) val newTableDesc = CatalogTable( @@ -86,7 +88,8 @@ case class CreateTableLikeCommand( schema = sourceTableDesc.schema, provider = newProvider, partitionColumnNames = sourceTableDesc.partitionColumnNames, - bucketSpec = sourceTableDesc.bucketSpec) + bucketSpec = sourceTableDesc.bucketSpec, + properties = properties) catalog.createTable(newTableDesc, ifNotExists) Seq.empty[Row] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZoneSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZoneSuite.scala index 90fb9e53d9f80..6a572c6c9dec4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZoneSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZoneSuite.scala @@ -69,14 +69,14 @@ abstract class BaseTimestampTableTimeZoneSuite extends SparkPlanTest with SQLTes } } - private def createRawData(spark: SparkSession): Dataset[(String, Timestamp)] = { + protected def createRawData(spark: SparkSession): Dataset[(String, Timestamp)] = { import spark.implicits._ val df = desiredTimestampStrings.toDF("display") // this will get the millis corresponding to the display time given the current session tz df.withColumn("ts", expr("cast(display as timestamp)")).as[(String, Timestamp)] } - private def checkHasTz(spark: SparkSession, table: String, tz: Option[String]): Unit = { + protected def checkHasTz(spark: SparkSession, table: String, tz: Option[String]): Unit = { val tableMetadata = spark.sessionState.catalog.getTableMetadata(TableIdentifier(table)) assert(tableMetadata.properties.get(TimestampTableTimeZone.TIMEZONE_PROPERTY) === tz, s"for table $table") @@ -337,8 +337,8 @@ abstract class BaseTimestampTableTimeZoneSuite extends SparkPlanTest with SQLTes withView("view_with_tz") { val exc1 = intercept[AnalysisException]{ spark.sql(s"""CREATE VIEW view_with_tz - | TBLPROPERTIES ("$key"="UTC") - | AS SELECT * FROM ok_table + |TBLPROPERTIES ("$key"="UTC") + |AS SELECT * FROM ok_table """.stripMargin) } assert(exc1.getMessage.contains("Timezone cannot be set for view")) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveTimestampTableTimeZoneSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveTimestampTableTimeZoneSuite.scala index f1cbd90d28acb..25252efda06af 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveTimestampTableTimeZoneSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveTimestampTableTimeZoneSuite.scala @@ -70,12 +70,12 @@ class HiveTimestampTableTimeZoneSuite extends BaseTimestampTableTimeZoneSuite // this isn't just a "ctas" sql statement b/c that doesn't let us specify the table tz spark.sql( s"""CREATE TABLE $dest ( - | display string, - | ts timestamp - |) - |STORED AS parquet - |$tblProperties - |""".stripMargin) + | display string, + | ts timestamp + |) + |STORED AS parquet + |$tblProperties + |""".stripMargin) spark.sql(s"insert into $dest select * from $source") true } else { @@ -84,4 +84,26 @@ class HiveTimestampTableTimeZoneSuite extends BaseTimestampTableTimeZoneSuite } } + + test("SPARK-12297: copy table timezone in CREATE TABLE LIKE") { + val key = TimestampTableTimeZone.TIMEZONE_PROPERTY + withTable("orig_hive", "copy_hive", "orig_ds", "copy_ds") { + spark.sql( + s"""CREATE TABLE orig_hive ( + | display string, + | ts timestamp + |) + |STORED AS parquet + |TBLPROPERTIES ("$key"="UTC") + |""". + stripMargin) + spark.sql("CREATE TABLE copy_hive LIKE orig_hive") + checkHasTz(spark, "copy_hive", Some("UTC")) + + createRawData(spark).write.option(key, "America/New_York").saveAsTable("orig_ds") + spark.sql("CREATE TABLE copy_ds LIKE orig_ds") + checkHasTz(spark, "copy_ds", Some("America/New_York")) + } + + } } From fbab47857f91cde7d7cf673406777e4afb818d2d Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 5 Oct 2017 16:19:31 -0700 Subject: [PATCH 11/23] Rename / move around a few things, plus grammar. --- .../expressions/datetimeExpressions.scala | 6 ++-- .../sql/catalyst/util/DateTimeUtils.scala | 26 +++++++++++++- .../apache/spark/sql/DataFrameReader.scala | 5 +-- .../apache/spark/sql/DataFrameWriter.scala | 18 ++++++---- .../spark/sql/execution/command/ddl.scala | 7 ++-- .../spark/sql/execution/command/tables.scala | 6 ++-- .../spark/sql/execution/command/views.scala | 4 +-- ...eTimeZone.scala => AdjustTimestamps.scala} | 35 ++++--------------- .../internal/BaseSessionStateBuilder.scala | 2 +- ...uite.scala => AdjustTimestampsSuite.scala} | 21 +++++------ .../sql/hive/HiveSessionStateBuilder.scala | 2 +- ....scala => HiveAdjustTimestampsSuite.scala} | 12 +++---- 12 files changed, 76 insertions(+), 68 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/{TimestampTableTimeZone.scala => AdjustTimestamps.scala} (86%) rename sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/{TimestampTableTimeZoneSuite.scala => AdjustTimestampsSuite.scala} (94%) rename sql/hive/src/test/scala/org/apache/spark/sql/hive/{HiveTimestampTableTimeZoneSuite.scala => HiveAdjustTimestampsSuite.scala} (88%) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index 463c7ffd4d911..cb16161942f11 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -1217,7 +1217,7 @@ case class ToUTCTimestamp(left: Expression, right: Expression) * another, for the same instant in time. * * We intentionally do not provide an ExpressionDescription as this is not meant to be exposed to - * users, its only used for internal conversions. + * users, it's only used for internal conversions. */ private[spark] case class TimestampTimezoneCorrection( time: Expression, @@ -1226,8 +1226,8 @@ private[spark] case class TimestampTimezoneCorrection( extends TernaryExpression with ImplicitCastInputTypes { // convertTz() does the *opposite* conversion we want, which is why from & to appear reversed - // in all the calls to convertTz. Its used for showing how the display time changes when we go - // from one timezone to another. We want to see how should change the SQLTimestamp value to + // in all the calls to convertTz. It's used for showing how the display time changes when we go + // from one timezone to another. We want to see how the SQLTimestamp value should change to // ensure the display does *not* change, despite going from one TZ to another. override def children: Seq[Expression] = Seq(time, from, to) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index 7f512b6042256..d1084317de95b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -26,6 +26,8 @@ import javax.xml.bind.DatatypeConverter import scala.annotation.tailrec +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.unsafe.types.UTF8String /** @@ -64,6 +66,7 @@ object DateTimeUtils { final val MonthOf31Days = Set(1, 3, 5, 7, 8, 10, 12) val TIMEZONE_OPTION = "timeZone" + val TIMEZONE_PROPERTY = "table.timezone-adjustment" def defaultTimeZone(): TimeZone = TimeZone.getDefault() @@ -109,7 +112,8 @@ object DateTimeUtils { computedTimeZones.computeIfAbsent(timeZoneId, computeTimeZone) } - lazy val validTimezones = TimeZone.getAvailableIDs().toSet + private lazy val validTimezones = TimeZone.getAvailableIDs().toSet + def isValidTimezone(timezoneId: String): Boolean = { validTimezones.contains(timezoneId) } @@ -1074,4 +1078,24 @@ object DateTimeUtils { threadLocalTimestampFormat.remove() threadLocalDateFormat.remove() } + + /** + * Throw an AnalysisException if we're trying to set an invalid timezone for this table. + */ + def checkTableTz(table: TableIdentifier, properties: Map[String, String]): Unit = { + checkTableTz(s"in table ${table.toString}", properties) + } + + /** + * Throw an AnalysisException if we're trying to set an invalid timezone for this table. + */ + def checkTableTz(dest: String, properties: Map[String, String]): Unit = { + properties.get(TIMEZONE_PROPERTY).foreach { tz => + if (!DateTimeUtils.isValidTimezone(tz)) { + throw new AnalysisException(s"Cannot set $TIMEZONE_PROPERTY to invalid " + + s"timezone $tz $dest") + } + } + } + } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 019208a3a3988..cde431f1b9882 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -27,8 +27,9 @@ import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.json.{CreateJacksonParser, JacksonParser, JSONOptions} +import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.command.DDLUtils -import org.apache.spark.sql.execution.datasources.{DataSource, FailureSafeParser, TimestampTableTimeZone} +import org.apache.spark.sql.execution.datasources.{DataSource, FailureSafeParser} import org.apache.spark.sql.execution.datasources.csv._ import org.apache.spark.sql.execution.datasources.jdbc._ import org.apache.spark.sql.execution.datasources.json.TextInputJsonDataSource @@ -179,7 +180,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { throw new AnalysisException("Hive data source can only be used with tables, you can not " + "read files of Hive data source directly.") } - TimestampTableTimeZone.checkTableTz("", extraOptions.toMap) + DateTimeUtils.checkTableTz("", extraOptions.toMap) sparkSession.baseRelationToDataFrame( DataSource.apply( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 760d28e5d92d5..133ede5a58d8a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -26,9 +26,10 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, UnresolvedRelation} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan} +import org.apache.spark.sql.catalyst.util.DateTimeUtils 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, TimestampTableTimeZone} +import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, LogicalRelation} import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.types.StructType @@ -215,7 +216,6 @@ 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() } @@ -231,6 +231,11 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { } assertNotBucketed("save") + val dest = extraOptions.get("path") match { + case Some(path) => s"for path $path" + case _ => s"with format $source" + } + DateTimeUtils.checkTableTz(dest, extraOptions.toMap) runCommand(df.sparkSession, "save") { DataSource( @@ -267,9 +272,9 @@ 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 => + extraOptions.get(DateTimeUtils.TIMEZONE_PROPERTY).foreach { tz => throw new AnalysisException("Cannot provide a table timezone on insert; tried to insert " + - s"$tableName with ${TimestampTableTimeZone.TIMEZONE_PROPERTY}=$tz") + s"$tableName with ${DateTimeUtils.TIMEZONE_PROPERTY}=$tz") } insertInto(df.sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName)) } @@ -411,9 +416,8 @@ 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 props = extraOptions.filterKeys(_ == DateTimeUtils.TIMEZONE_PROPERTY).toMap + DateTimeUtils.checkTableTz(tableIdent, props) val tableDesc = CatalogTable( identifier = tableIdent, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 6625f04f0ba1d..ead034f554846 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -34,7 +34,8 @@ 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, TimestampTableTimeZone} +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, PartitioningUtils} 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 @@ -231,11 +232,11 @@ case class AlterTableSetPropertiesCommand( extends RunnableCommand { if (isView) { - properties.get(TimestampTableTimeZone.TIMEZONE_PROPERTY).foreach { _ => + properties.get(DateTimeUtils.TIMEZONE_PROPERTY).foreach { _ => throw new AnalysisException("Timezone cannot be set for view") } } - TimestampTableTimeZone.checkTableTz(tableName, properties) + DateTimeUtils.checkTableTz(tableName, properties) override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index ec3e90d64ccb8..c3e6ba281bdf6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTableType._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.util.{quoteIdentifier, DateTimeUtils} -import org.apache.spark.sql.execution.datasources.{DataSource, PartitioningUtils, TimestampTableTimeZone} +import org.apache.spark.sql.execution.datasources.{DataSource, PartitioningUtils} import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat import org.apache.spark.sql.execution.datasources.json.JsonFileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat @@ -77,7 +77,7 @@ case class CreateTableLikeCommand( // Otherwise create a managed table. val tblType = if (location.isEmpty) CatalogTableType.MANAGED else CatalogTableType.EXTERNAL val properties = - sourceTableDesc.properties.filterKeys(_ == TimestampTableTimeZone.TIMEZONE_PROPERTY) + sourceTableDesc.properties.filterKeys(_ == DateTimeUtils.TIMEZONE_PROPERTY) val newTableDesc = CatalogTable( @@ -130,7 +130,7 @@ case class CreateTableCommand( Seq.empty[Row] } - TimestampTableTimeZone.checkTableTz(table.identifier, table.properties) + DateTimeUtils.checkTableTz(table.identifier, table.properties) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index 593dae1a93925..ab6f9c237ab01 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable import org.apache.spark.sql.catalyst.expressions.{Alias, SubqueryExpression} import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, View} -import org.apache.spark.sql.execution.datasources.TimestampTableTimeZone +import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types.MetadataBuilder import org.apache.spark.sql.util.SchemaUtils @@ -124,7 +124,7 @@ case class CreateViewCommand( s"It is not allowed to add database prefix `$database` for the TEMPORARY view name.") } - properties.get(TimestampTableTimeZone.TIMEZONE_PROPERTY).foreach { _ => + properties.get(DateTimeUtils.TIMEZONE_PROPERTY).foreach { _ => throw new AnalysisException("Timezone cannot be set for view") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZone.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/AdjustTimestamps.scala similarity index 86% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZone.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/AdjustTimestamps.scala index edf02415d5ce9..b190d9db71f84 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZone.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/AdjustTimestamps.scala @@ -27,12 +27,12 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types.{StringType, TimestampType} /** - * Apply a correction to data loaded from, or saved to, Parquet, so that it timestamps can be read - * like TIMESTAMP WITHOUT TIMEZONE. This gives correct behavior if you process data with - * machines in different timezones, or if you access the data from multiple SQL engines. + * Apply a correction to data loaded from, or saved to, tables that have a configured time zone, so + * that timestamps can be read like TIMESTAMP WITHOUT TIMEZONE. This gives correct behavior if you + * process data with machines in different timezones, or if you access the data from multiple SQL + * engines. */ -private[sql] case class TimestampTableTimeZone(sparkSession: SparkSession) - extends Rule[LogicalPlan] { +private[sql] case class AdjustTimestamps(sparkSession: SparkSession) extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = { // we can't use transformUp because we want to terminate recursion if there was already @@ -138,7 +138,7 @@ private[sql] case class TimestampTableTimeZone(sparkSession: SparkSession) } private def extractTableTz(options: Map[String, String]): Option[String] = { - options.get(TimestampTableTimeZone.TIMEZONE_PROPERTY) + options.get(DateTimeUtils.TIMEZONE_PROPERTY) } private def extractTableTz( @@ -188,26 +188,3 @@ private[sql] case class TimestampTableTimeZone(sparkSession: SparkSession) (foundTs, modifiedFields, replacements) } } - -private[sql] object TimestampTableTimeZone { - val TIMEZONE_PROPERTY = "table.timezone-adjustment" - - /** - * Throw an AnalysisException if we're trying to set an invalid timezone for this table. - */ - private[sql] def checkTableTz(table: TableIdentifier, properties: Map[String, String]): Unit = { - checkTableTz(s"in table ${table.toString}", properties) - } - - /** - * Throw an AnalysisException if we're trying to set an invalid timezone for this table. - */ - private[sql] def checkTableTz(dest: String, properties: Map[String, String]): Unit = { - properties.get(TIMEZONE_PROPERTY).foreach { tz => - if (!DateTimeUtils.isValidTimezone(tz)) { - throw new AnalysisException(s"Cannot set $TIMEZONE_PROPERTY to invalid " + - s"timezone $tz $dest") - } - } - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala index efedbcf774b5f..d7b027c49292a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala @@ -164,7 +164,7 @@ abstract class BaseSessionStateBuilder( PreprocessTableCreation(session) +: PreprocessTableInsertion(conf) +: DataSourceAnalysis(conf) +: - TimestampTableTimeZone(session) +: + AdjustTimestamps(session) +: customPostHocResolutionRules override val extendedCheckRules: Seq[LogicalPlan => Unit] = diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZoneSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/AdjustTimestampsSuite.scala similarity index 94% rename from sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZoneSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/AdjustTimestampsSuite.scala index 6a572c6c9dec4..ae75a12709ba7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/TimestampTableTimeZoneSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/AdjustTimestampsSuite.scala @@ -24,11 +24,12 @@ import org.scalatest.BeforeAndAfterAll import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.SparkPlanTest import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} -abstract class BaseTimestampTableTimeZoneSuite extends SparkPlanTest with SQLTestUtils +abstract class BaseAdjustTimestampsSuite extends SparkPlanTest with SQLTestUtils with BeforeAndAfterAll { var originalTz: TimeZone = _ @@ -78,7 +79,7 @@ abstract class BaseTimestampTableTimeZoneSuite extends SparkPlanTest with SQLTes protected def checkHasTz(spark: SparkSession, table: String, tz: Option[String]): Unit = { val tableMetadata = spark.sessionState.catalog.getTableMetadata(TableIdentifier(table)) - assert(tableMetadata.properties.get(TimestampTableTimeZone.TIMEZONE_PROPERTY) === tz, + assert(tableMetadata.properties.get(DateTimeUtils.TIMEZONE_PROPERTY) === tz, s"for table $table") } @@ -126,7 +127,7 @@ abstract class BaseTimestampTableTimeZoneSuite extends SparkPlanTest with SQLTes tz: Option[String], format: String): Boolean = { val writer = df.write.format(format) - tz.foreach { writer.option(TimestampTableTimeZone.TIMEZONE_PROPERTY, _)} + tz.foreach { writer.option(DateTimeUtils.TIMEZONE_PROPERTY, _)} writer.saveAsTable(table) true } @@ -151,7 +152,7 @@ abstract class BaseTimestampTableTimeZoneSuite extends SparkPlanTest with SQLTes destTz: Option[String], destFormat: String): Boolean = { val writer = spark.sql(s"select * from $source").write.format(destFormat) - destTz.foreach { writer.option(TimestampTableTimeZone.TIMEZONE_PROPERTY, _)} + destTz.foreach { writer.option(DateTimeUtils.TIMEZONE_PROPERTY, _)} writer.saveAsTable(dest) true } @@ -161,7 +162,7 @@ abstract class BaseTimestampTableTimeZoneSuite extends SparkPlanTest with SQLTes test("SPARK-12297: Read and write with table timezones") { assert(TimeZone.getDefault.getID() === "America/Los_Angeles") - val key = TimestampTableTimeZone.TIMEZONE_PROPERTY + val key = DateTimeUtils.TIMEZONE_PROPERTY val originalData = createRawData(spark) withTempPath { basePath => val dsPath = new File(basePath, "dsFlat").getAbsolutePath @@ -190,7 +191,7 @@ abstract class BaseTimestampTableTimeZoneSuite extends SparkPlanTest with SQLTes // values have been adjusted as we expect. val tableMeta = spark.sessionState.catalog.getTableMetadata(TableIdentifier(table)) val location = tableMeta.location.toString() - val tz = tableMeta.properties.get(TimestampTableTimeZone.TIMEZONE_PROPERTY) + val tz = tableMeta.properties.get(DateTimeUtils.TIMEZONE_PROPERTY) // some formats need the schema specified val df = spark.read.schema(originalData.schema).format(format).load(location) checkRawData(df, tz.getOrElse("America/Los_Angeles")) @@ -280,7 +281,7 @@ abstract class BaseTimestampTableTimeZoneSuite extends SparkPlanTest with SQLTes test("SPARK-12297: exception on bad timezone") { // make sure there is an exception anytime we try to read or write with a bad timezone - val key = TimestampTableTimeZone.TIMEZONE_PROPERTY + val key = DateTimeUtils.TIMEZONE_PROPERTY val badVal = "Blart Versenwald III" val data = createRawData(spark) def hasBadTzException(command: => Unit): Unit = { @@ -321,7 +322,7 @@ abstract class BaseTimestampTableTimeZoneSuite extends SparkPlanTest with SQLTes val origData = createRawData(spark) origData.write.saveAsTable("some_table") val exc = intercept[AnalysisException]{ - createRawData(spark).write.option(TimestampTableTimeZone.TIMEZONE_PROPERTY, "UTC") + createRawData(spark).write.option(DateTimeUtils.TIMEZONE_PROPERTY, "UTC") .insertInto("some_table") } assert(exc.getMessage.contains("Cannot provide a table timezone on insert")) @@ -329,7 +330,7 @@ abstract class BaseTimestampTableTimeZoneSuite extends SparkPlanTest with SQLTes } test("SPARK-12297: refuse table timezone on views") { - val key = TimestampTableTimeZone.TIMEZONE_PROPERTY + val key = DateTimeUtils.TIMEZONE_PROPERTY val originalData = createRawData(spark) withTable("ok_table") { @@ -352,4 +353,4 @@ abstract class BaseTimestampTableTimeZoneSuite extends SparkPlanTest with SQLTes } } -class TimestampTableTimeZoneSuite extends BaseTimestampTableTimeZoneSuite with SharedSQLContext +class AdjustTimestampsSuite extends BaseAdjustTimestampsSuite with SharedSQLContext diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala index 987ec5da3376b..b3b5ffb197451 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala @@ -80,7 +80,7 @@ class HiveSessionStateBuilder(session: SparkSession, parentState: Option[Session PreprocessTableInsertion(conf) +: DataSourceAnalysis(conf) +: HiveAnalysis +: - TimestampTableTimeZone(session) +: + AdjustTimestamps(session) +: customPostHocResolutionRules override val extendedCheckRules: Seq[LogicalPlan => Unit] = diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveTimestampTableTimeZoneSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveAdjustTimestampsSuite.scala similarity index 88% rename from sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveTimestampTableTimeZoneSuite.scala rename to sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveAdjustTimestampsSuite.scala index 25252efda06af..46c6a8fc27166 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveTimestampTableTimeZoneSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveAdjustTimestampsSuite.scala @@ -17,11 +17,11 @@ package org.apache.spark.sql.hive import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.execution.datasources.{BaseTimestampTableTimeZoneSuite, TimestampTableTimeZone} +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.execution.datasources.{AdjustTimestamps, BaseAdjustTimestampsSuite} import org.apache.spark.sql.hive.test.TestHiveSingleton -class HiveTimestampTableTimeZoneSuite extends BaseTimestampTableTimeZoneSuite - with TestHiveSingleton { +class HiveAdjustTimestampsSuite extends BaseAdjustTimestampsSuite with TestHiveSingleton { override protected def createAndSaveTableFunctions(): Seq[CreateAndSaveTable] = { super.createAndSaveTableFunctions() ++ Seq(CreateHiveTableAndInsert) @@ -39,7 +39,7 @@ class HiveTimestampTableTimeZoneSuite extends BaseTimestampTableTimeZoneSuite format: String): Boolean = { if (format == "parquet") { val tblProperties = tzOpt.map { tz => - s"""TBLPROPERTIES ("${TimestampTableTimeZone.TIMEZONE_PROPERTY}"="$tz")""" + s"""TBLPROPERTIES ("${DateTimeUtils.TIMEZONE_PROPERTY}"="$tz")""" }.getOrElse("") spark.sql( s"""CREATE TABLE $table ( @@ -65,7 +65,7 @@ class HiveTimestampTableTimeZoneSuite extends BaseTimestampTableTimeZoneSuite destFormat: String): Boolean = { if (destFormat == "parquet") { val tblProperties = destTz.map { tz => - s"""TBLPROPERTIES ("${TimestampTableTimeZone.TIMEZONE_PROPERTY}"="$tz")""" + s"""TBLPROPERTIES ("${DateTimeUtils.TIMEZONE_PROPERTY}"="$tz")""" }.getOrElse("") // this isn't just a "ctas" sql statement b/c that doesn't let us specify the table tz spark.sql( @@ -86,7 +86,7 @@ class HiveTimestampTableTimeZoneSuite extends BaseTimestampTableTimeZoneSuite } test("SPARK-12297: copy table timezone in CREATE TABLE LIKE") { - val key = TimestampTableTimeZone.TIMEZONE_PROPERTY + val key = DateTimeUtils.TIMEZONE_PROPERTY withTable("orig_hive", "copy_hive", "orig_ds", "copy_ds") { spark.sql( s"""CREATE TABLE orig_hive ( From 964af6ddaf4de304a2d1f39a281cdfa2bd2f30ee Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 5 Oct 2017 17:07:30 -0700 Subject: [PATCH 12/23] Simplify some code. --- .../datasources/AdjustTimestamps.scala | 40 ++++++------------- 1 file changed, 13 insertions(+), 27 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/AdjustTimestamps.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/AdjustTimestamps.scala index b190d9db71f84..16468a72d5053 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/AdjustTimestamps.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/AdjustTimestamps.scala @@ -63,7 +63,7 @@ private[sql] case class AdjustTimestamps(sparkSession: SparkSession) extends Rul (alreadyConverted, Map()) case lr@LogicalRelation(fsRelation: HadoopFsRelation, _, _, _) => val tzOpt = extractTableTz(lr.catalogTable, fsRelation.options) - tzOpt.map { tableTz => + tzOpt.flatMap { tableTz => // the table has a timezone set, so after reading the data, apply a conversion // SessionTZ (instead of JVM TZ) will make the time display correctly in SQL queries, but @@ -72,17 +72,13 @@ private[sql] case class AdjustTimestamps(sparkSession: SparkSession) extends Rul if (toTz != tableTz) { logDebug(s"table tz = $tableTz; converting to current session tz = $toTz") // find timestamp columns, and convert their tz - val (foundTs, modifiedFields, replacements) = - convertTzForAllTimestamps(lr, tableTz, toTz) - if (foundTs) { - (new Project(modifiedFields, lr), replacements) - } else { - (lr, Map[ExprId, NamedExpression]()) + convertTzForAllTimestamps(lr, tableTz, toTz).map { case (fields, replacements) => + (new Project(fields, lr), replacements) } } else { - (lr, Map[ExprId, NamedExpression]()) + None } - }.getOrElse((lr, Map[ExprId, NamedExpression]())) + }.getOrElse((lr, Map())) case other => // first, process all the children -- this ensures we have the right renames in scope. var newReplacements = Map[ExprId, NamedExpression]() @@ -107,16 +103,10 @@ private[sql] case class AdjustTimestamps(sparkSession: SparkSession) extends Rul (fixedExpressions, newReplacements) } - private def hasCorrection(exprs: Seq[NamedExpression]): Boolean = { - var hasCorrection = false - exprs.foreach { expr => - expr.foreach { - case _: TimestampTimezoneCorrection => - hasCorrection = true - case other => // no-op - } + private def hasCorrection(exprs: Seq[Expression]): Boolean = { + exprs.exists { expr => + expr.isInstanceOf[TimestampTimezoneCorrection] || hasCorrection(expr.children) } - hasCorrection } private def writeConversion( @@ -125,13 +115,9 @@ private[sql] case class AdjustTimestamps(sparkSession: SparkSession) extends Rul val tableTz = extractTableTz(insertIntoHadoopFs.catalogTable, insertIntoHadoopFs.options) val internalTz = sparkSession.sessionState.conf.sessionLocalTimeZone if (tableTz.isDefined && tableTz != internalTz) { - val (foundTsFields, modifiedFields, _) = - convertTzForAllTimestamps(query, internalTz, tableTz.get) - if (foundTsFields) { - insertIntoHadoopFs.copy(query = new Project(modifiedFields, query)) - } else { - insertIntoHadoopFs - } + convertTzForAllTimestamps(query, internalTz, tableTz.get).map { case (fields, _) => + insertIntoHadoopFs.copy(query = new Project(fields, query)) + }.getOrElse(insertIntoHadoopFs) } else { insertIntoHadoopFs } @@ -156,7 +142,7 @@ private[sql] case class AdjustTimestamps(sparkSession: SparkSession) extends Rul private def convertTzForAllTimestamps( relation: LogicalPlan, fromTz: String, - toTz: String): (Boolean, Seq[NamedExpression], Map[ExprId, NamedExpression]) = { + toTz: String): Option[(Seq[NamedExpression], Map[ExprId, NamedExpression])] = { val schema = relation.schema var foundTs = false var replacements = Map[ExprId, NamedExpression]() @@ -185,6 +171,6 @@ private[sql] case class AdjustTimestamps(sparkSession: SparkSession) extends Rul exp } } - (foundTs, modifiedFields, replacements) + if (foundTs) Some((modifiedFields, replacements)) else None } } From 0b4ba84c2e2ec8b46fb187b8d1474e518c248552 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 6 Oct 2017 12:56:33 -0700 Subject: [PATCH 13/23] Cache TZ objects in generated code. --- .../expressions/datetimeExpressions.scala | 16 +++++++++++----- .../spark/sql/catalyst/util/DateTimeUtils.scala | 4 ---- 2 files changed, 11 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index cb16161942f11..4a993846d6661 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -1238,8 +1238,8 @@ private[spark] case class TimestampTimezoneCorrection( override def nullSafeEval(time: Any, from: Any, to: Any): Any = { DateTimeUtils.convertTz( time.asInstanceOf[Long], - to.asInstanceOf[UTF8String].toString(), - from.asInstanceOf[UTF8String].toString()) + DateTimeUtils.getTimeZone(to.asInstanceOf[UTF8String].toString()), + DateTimeUtils.getTimeZone(from.asInstanceOf[UTF8String].toString())) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { @@ -1253,14 +1253,20 @@ private[spark] case class TimestampTimezoneCorrection( |long ${ev.value} = 0; """.stripMargin) } else { + val fromTerm = ctx.freshName("from") + val toTerm = ctx.freshName("to") + val tzClass = classOf[TimeZone].getName val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") + ctx.addMutableState(tzClass, fromTerm, s"""$fromTerm = $dtu.getTimeZone("$fromTz");""") + ctx.addMutableState(tzClass, toTerm, s"""$toTerm = $dtu.getTimeZone("$toTz");""") + val eval = time.genCode(ctx) ev.copy(code = s""" |${eval.code} |boolean ${ev.isNull} = ${eval.isNull}; |long ${ev.value} = 0; |if (!${ev.isNull}) { - | ${ev.value} = $dtu.convertTz(${eval.value}, "$toTz", "$fromTz"); + | ${ev.value} = $dtu.convertTz(${eval.value}, $toTerm, $fromTerm); |} """.stripMargin) } @@ -1269,8 +1275,8 @@ private[spark] case class TimestampTimezoneCorrection( s""" |${ev.value} = $dtu.convertTz( | $time, - | $to.toString(), - | $from.toString()); + | $dtu.getTimeZone($to.toString()), + | $dtu.getTimeZone($from.toString())); """.stripMargin ) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index d1084317de95b..acf09993fd38f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -1024,10 +1024,6 @@ 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. * From d5086c236be55171d19cf7dc13eb5fc411e5aff6 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 6 Oct 2017 13:02:38 -0700 Subject: [PATCH 14/23] Add comment about the value of TIMEZONE_PROPERTY. --- .../org/apache/spark/sql/catalyst/util/DateTimeUtils.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index acf09993fd38f..173a66c9bbfff 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -66,6 +66,12 @@ object DateTimeUtils { final val MonthOf31Days = Set(1, 3, 5, 7, 8, 10, 12) val TIMEZONE_OPTION = "timeZone" + + /** + * Property that holds the time zone used for adjusting "timestamp without time zone" + * columns to the session's time zone. See SPARK-12297 for more details (including the + * specified name of this property). + */ val TIMEZONE_PROPERTY = "table.timezone-adjustment" def defaultTimeZone(): TimeZone = TimeZone.getDefault() From 0b2aaeffb258e778623e3f364d28145ae5336b35 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 6 Oct 2017 15:38:13 -0700 Subject: [PATCH 15/23] Add test with convertMetastoreParquet=false. This triggers use of InsertIntoHiveTable. Tests currently fail because that command is not yet processed by the new timestamp code. --- .../datasources/AdjustTimestampsSuite.scala | 2 +- .../sql/hive/HiveAdjustTimestampsSuite.scala | 35 +++++++++++-------- 2 files changed, 22 insertions(+), 15 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/AdjustTimestampsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/AdjustTimestampsSuite.scala index ae75a12709ba7..2bb9ab1d932d8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/AdjustTimestampsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/AdjustTimestampsSuite.scala @@ -127,7 +127,7 @@ abstract class BaseAdjustTimestampsSuite extends SparkPlanTest with SQLTestUtils tz: Option[String], format: String): Boolean = { val writer = df.write.format(format) - tz.foreach { writer.option(DateTimeUtils.TIMEZONE_PROPERTY, _)} + tz.foreach(writer.option(DateTimeUtils.TIMEZONE_PROPERTY, _)) writer.saveAsTable(table) true } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveAdjustTimestampsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveAdjustTimestampsSuite.scala index 46c6a8fc27166..753fef31d179f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveAdjustTimestampsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveAdjustTimestampsSuite.scala @@ -24,20 +24,26 @@ import org.apache.spark.sql.hive.test.TestHiveSingleton class HiveAdjustTimestampsSuite extends BaseAdjustTimestampsSuite with TestHiveSingleton { override protected def createAndSaveTableFunctions(): Seq[CreateAndSaveTable] = { - super.createAndSaveTableFunctions() ++ Seq(CreateHiveTableAndInsert) + super.createAndSaveTableFunctions() ++ + Seq(true, false).map(new CreateHiveTableAndInsert(_)) } override protected def ctasFunctions(): Seq[CTAS] = { - super.ctasFunctions() ++ Seq(CreateHiveTableWithTimezoneAndInsert) + super.ctasFunctions() ++ + Seq(true, false).map(new CreateHiveTableWithTimezoneAndInsert(_)) } - object CreateHiveTableAndInsert extends CreateAndSaveTable { + class CreateHiveTableAndInsert(convertMetastore: Boolean) extends CreateAndSaveTable { override def createAndSave( df: DataFrame, table: String, tzOpt: Option[String], format: String): Boolean = { - if (format == "parquet") { + if (format != "parquet") { + return false + } + + withSQLConf(HiveUtils.CONVERT_METASTORE_PARQUET.key -> convertMetastore.toString) { val tblProperties = tzOpt.map { tz => s"""TBLPROPERTIES ("${DateTimeUtils.TIMEZONE_PROPERTY}"="$tz")""" }.getOrElse("") @@ -46,24 +52,27 @@ class HiveAdjustTimestampsSuite extends BaseAdjustTimestampsSuite with TestHiveS | display string, | ts timestamp |) - |STORED AS parquet + |STORED AS $format |$tblProperties |""".stripMargin) df.write.insertInto(table) - true - } else { - false } + + true } } - object CreateHiveTableWithTimezoneAndInsert extends CTAS { + class CreateHiveTableWithTimezoneAndInsert(convertMetastore: Boolean) extends CTAS { override def createTableFromSourceTable( source: String, dest: String, destTz: Option[String], destFormat: String): Boolean = { - if (destFormat == "parquet") { + if (destFormat != "parquet") { + return false + } + + withSQLConf(HiveUtils.CONVERT_METASTORE_PARQUET.key -> convertMetastore.toString) { val tblProperties = destTz.map { tz => s"""TBLPROPERTIES ("${DateTimeUtils.TIMEZONE_PROPERTY}"="$tz")""" }.getOrElse("") @@ -73,15 +82,13 @@ class HiveAdjustTimestampsSuite extends BaseAdjustTimestampsSuite with TestHiveS | display string, | ts timestamp |) - |STORED AS parquet + |STORED AS $destFormat |$tblProperties |""".stripMargin) spark.sql(s"insert into $dest select * from $source") - true - } else { - false } + true } } From 63016db7bef3c1ecd0b9c5d0d98c6ed8a7beb9f5 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 6 Oct 2017 15:54:22 -0700 Subject: [PATCH 16/23] Add adjustment rule for InsertIntoHiveTable. Tests still don't pass. Kinda hard to figure out which test is failing with current code. --- .../datasources/AdjustTimestamps.scala | 79 ++++++++++--------- .../sql/hive/HiveSessionStateBuilder.scala | 1 + .../spark/sql/hive/HiveStrategies.scala | 29 ++++++- 3 files changed, 72 insertions(+), 37 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/AdjustTimestamps.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/AdjustTimestamps.scala index 16468a72d5053..d8ba4744d70c3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/AdjustTimestamps.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/AdjustTimestamps.scala @@ -26,37 +26,14 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types.{StringType, TimestampType} -/** - * Apply a correction to data loaded from, or saved to, tables that have a configured time zone, so - * that timestamps can be read like TIMESTAMP WITHOUT TIMEZONE. This gives correct behavior if you - * process data with machines in different timezones, or if you access the data from multiple SQL - * engines. - */ -private[sql] case class AdjustTimestamps(sparkSession: SparkSession) extends Rule[LogicalPlan] { - - def apply(plan: LogicalPlan): LogicalPlan = { - // we can't use transformUp because we want to terminate recursion if there was already - // timestamp correction, to keep this idempotent. - plan match { - case insertIntoHadoopFs: InsertIntoHadoopFsRelationCommand => - // The query might be reading from a parquet table which requires a different conversion; - // this makes sure we apply the correct conversions there. - val (fixedQuery, _) = convertInputs(insertIntoHadoopFs.query) - writeConversion(insertIntoHadoopFs.copy(query = fixedQuery)) - - case other => - // recurse into children to see if we're reading data that needs conversion - val (convertedPlan, _) = convertInputs(plan) - convertedPlan - } - } +abstract class BaseAdjustTimestampsRule(sparkSession: SparkSession) extends Rule[LogicalPlan] { /** * Apply the correction to all timestamp inputs, and replace all references to the raw attributes * with the new converted inputs. * @return The converted plan, and the replacements to be applied further up the plan */ - private def convertInputs( + protected def convertInputs( plan: LogicalPlan ): (LogicalPlan, Map[ExprId, NamedExpression]) = plan match { case alreadyConverted@Project(exprs, _) if hasCorrection(exprs) => @@ -103,31 +80,32 @@ private[sql] case class AdjustTimestamps(sparkSession: SparkSession) extends Rul (fixedExpressions, newReplacements) } - private def hasCorrection(exprs: Seq[Expression]): Boolean = { + protected def hasCorrection(exprs: Seq[Expression]): Boolean = { exprs.exists { expr => expr.isInstanceOf[TimestampTimezoneCorrection] || hasCorrection(expr.children) } } - private def writeConversion( - insertIntoHadoopFs: InsertIntoHadoopFsRelationCommand): InsertIntoHadoopFsRelationCommand = { - val query = insertIntoHadoopFs.query - val tableTz = extractTableTz(insertIntoHadoopFs.catalogTable, insertIntoHadoopFs.options) + protected def writeConversion( + table: Option[CatalogTable], + options: Map[String, String], + query: LogicalPlan): LogicalPlan = { + val tableTz = extractTableTz(table, options) val internalTz = sparkSession.sessionState.conf.sessionLocalTimeZone if (tableTz.isDefined && tableTz != internalTz) { convertTzForAllTimestamps(query, internalTz, tableTz.get).map { case (fields, _) => - insertIntoHadoopFs.copy(query = new Project(fields, query)) - }.getOrElse(insertIntoHadoopFs) + new Project(fields, query) + }.getOrElse(query) } else { - insertIntoHadoopFs + query } } - private def extractTableTz(options: Map[String, String]): Option[String] = { + protected def extractTableTz(options: Map[String, String]): Option[String] = { options.get(DateTimeUtils.TIMEZONE_PROPERTY) } - private def extractTableTz( + protected def extractTableTz( table: Option[CatalogTable], options: Map[String, String]): Option[String] = { table.flatMap { tbl => extractTableTz(tbl.properties) }.orElse(extractTableTz(options)) @@ -139,7 +117,7 @@ private[sql] case class AdjustTimestamps(sparkSession: SparkSession) extends Rul * (Leave non-timestamp fields alone.) Also return a map from the original id for the timestamp * field, to the new alias of the timezone-corrected expression. */ - private def convertTzForAllTimestamps( + protected def convertTzForAllTimestamps( relation: LogicalPlan, fromTz: String, toTz: String): Option[(Seq[NamedExpression], Map[ExprId, NamedExpression])] = { @@ -174,3 +152,32 @@ private[sql] case class AdjustTimestamps(sparkSession: SparkSession) extends Rul if (foundTs) Some((modifiedFields, replacements)) else None } } + +/** + * Apply a correction to data loaded from, or saved to, tables that have a configured time zone, so + * that timestamps can be read like TIMESTAMP WITHOUT TIMEZONE. This gives correct behavior if you + * process data with machines in different timezones, or if you access the data from multiple SQL + * engines. + */ +case class AdjustTimestamps(sparkSession: SparkSession) + extends BaseAdjustTimestampsRule(sparkSession) { + + def apply(plan: LogicalPlan): LogicalPlan = { + // we can't use transformUp because we want to terminate recursion if there was already + // timestamp correction, to keep this idempotent. + plan match { + case insert: InsertIntoHadoopFsRelationCommand => + // The query might be reading from a parquet table which requires a different conversion; + // this makes sure we apply the correct conversions there. + val (fixedQuery, _) = convertInputs(insert.query) + val fixedOutput = writeConversion(insert.catalogTable, insert.options, insert.query) + insert.copy(query = fixedOutput) + + case other => + // recurse into children to see if we're reading data that needs conversion + val (convertedPlan, _) = convertInputs(plan) + convertedPlan + } + } + +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala index b3b5ffb197451..2a02375dc9e85 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala @@ -80,6 +80,7 @@ class HiveSessionStateBuilder(session: SparkSession, parentState: Option[Session PreprocessTableInsertion(conf) +: DataSourceAnalysis(conf) +: HiveAnalysis +: + HiveAdjustTimestamps(session) +: AdjustTimestamps(session) +: customPostHocResolutionRules diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 805b3171cdaab..11463f61f5347 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoDir, InsertIntoTab import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.command.{CreateTableCommand, DDLUtils} -import org.apache.spark.sql.execution.datasources.{CreateTable, LogicalRelation} +import org.apache.spark.sql.execution.datasources.{BaseAdjustTimestampsRule, CreateTable, LogicalRelation} import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, ParquetOptions} import org.apache.spark.sql.hive.execution._ import org.apache.spark.sql.hive.orc.OrcFileFormat @@ -217,6 +217,33 @@ case class RelationConversions( } } +/** + * Apply a correction to data loaded from, or saved to, tables that have a configured time zone, so + * that timestamps can be read like TIMESTAMP WITHOUT TIMEZONE. This gives correct behavior if you + * process data with machines in different timezones, or if you access the data from multiple SQL + * engines. + */ +case class HiveAdjustTimestamps(sparkSession: SparkSession) + extends BaseAdjustTimestampsRule(sparkSession) { + + def apply(plan: LogicalPlan): LogicalPlan = { + // we can't use transformUp because we want to terminate recursion if there was already + // timestamp correction, to keep this idempotent. + plan match { + case insert: InsertIntoHiveTable => + // The query might be reading from a parquet table which requires a different conversion; + // this makes sure we apply the correct conversions there. + val (fixedQuery, _) = convertInputs(insert.query) + val fixedOutput = writeConversion(Some(insert.table), Map(), insert.query) + insert.copy(query = fixedOutput) + + case other => + plan + } + } + +} + private[hive] trait HiveStrategies { // Possibly being too clever with types here... or not clever enough. self: SparkPlanner => From 7ee9c8a3ed743923538f116c0e26095c1af2c836 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 6 Oct 2017 16:25:14 -0700 Subject: [PATCH 17/23] Break out tests into individual combinations. This makes it easier to see what combinations fail, even if the suite itself ends up running for a bit longer. Doesn't seem bad, though. --- .../datasources/AdjustTimestampsSuite.scala | 194 ++++++++---------- .../sql/hive/HiveAdjustTimestampsSuite.scala | 50 ++--- 2 files changed, 109 insertions(+), 135 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/AdjustTimestampsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/AdjustTimestampsSuite.scala index 2bb9ab1d932d8..e0610e41790fb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/AdjustTimestampsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/AdjustTimestampsSuite.scala @@ -99,68 +99,67 @@ abstract class BaseAdjustTimestampsSuite extends SparkPlanTest with SQLTestUtils } } + private val formats = Seq("parquet", "csv", "json") + // we want to test that this works w/ hive-only methods as well, so provide a few extension // points so we can also easily re-use this with hive support. - protected def createAndSaveTableFunctions(): Seq[CreateAndSaveTable] = { - Seq(CreateAndSaveDatasourceTable) + protected def createAndSaveTableFunctions(): Map[String, CreateAndSaveTable] = { + formats.map { format => + (format, new CreateAndSaveDatasourceTable(format)) + }.toMap } - protected def ctasFunctions(): Seq[CTAS] = { - Seq(DatasourceCTAS) + + protected def ctasFunctions(): Map[String, CTAS] = { + formats.map { format => + (format, new DatasourceCTAS(format)) + }.toMap } trait CreateAndSaveTable { - /** - * if the format is unsupported return false (and do nothing else). - * otherwise, create the table, save the dataset into it, and return true - */ - def createAndSave( - ds: DataFrame, - table: String, - tz: Option[String], - format: String): Boolean + /** Create the table and save the contents of the dataset into it. */ + def createAndSave(df: DataFrame, table: String, tz: Option[String]): Unit + + /** The target table's format. */ + val format: String } - object CreateAndSaveDatasourceTable extends CreateAndSaveTable { - override def createAndSave( - df: DataFrame, - table: String, - tz: Option[String], - format: String): Boolean = { + class CreateAndSaveDatasourceTable(override val format: String) extends CreateAndSaveTable { + override def createAndSave(df: DataFrame, table: String, tz: Option[String]): Unit = { val writer = df.write.format(format) tz.foreach(writer.option(DateTimeUtils.TIMEZONE_PROPERTY, _)) writer.saveAsTable(table) - true } } trait CTAS { /** - * If the format is unsupported, return false (and do nothing else). Otherwise, create a table - * with the given time zone, and copy the entire contents of another table into it. + * Create a table with the given time zone, and copy the entire contents of the source table + * into it. */ - def createTableFromSourceTable( - source: String, - dest: String, - destTz: Option[String], - destFormat: String): Boolean + def createFromSource(source: String, dest: String, destTz: Option[String]): Unit } - object DatasourceCTAS extends CTAS { - override def createTableFromSourceTable( - source: String, - dest: String, - destTz: Option[String], - destFormat: String): Boolean = { - val writer = spark.sql(s"select * from $source").write.format(destFormat) + class DatasourceCTAS(format: String) extends CTAS { + override def createFromSource(source: String, dest: String, destTz: Option[String]): Unit = { + val writer = spark.sql(s"select * from $source").write.format(format) destTz.foreach { writer.option(DateTimeUtils.TIMEZONE_PROPERTY, _)} writer.saveAsTable(dest) - true } } - val formats = Seq("parquet", "csv", "json") + createAndSaveTableFunctions().foreach { case (fmt, createFn) => + ctasFunctions().foreach { case (destFmt, ctasFn) => + test(s"timestamp adjustment: in=$fmt, out=$destFmt") { + testTimestampAdjustment(fmt, destFmt, createFn, ctasFn) + } + } + } - test("SPARK-12297: Read and write with table timezones") { + private def testTimestampAdjustment( + format: String, + destFormat: String, + createFn: CreateAndSaveTable, + ctasFn: CTAS): Unit = { assert(TimeZone.getDefault.getID() === "America/Los_Angeles") val key = DateTimeUtils.TIMEZONE_PROPERTY val originalData = createRawData(spark) @@ -212,74 +211,61 @@ abstract class BaseAdjustTimestampsSuite extends SparkPlanTest with SQLTestUtils // save to tables, and read the data back -- this time, the timezone conversion should be // automatic from the table metadata, we don't need to supply any options when reading the // data. Works across different ways of creating the tables and different data formats. - createAndSaveTableFunctions().foreach { createAndSave => - formats.foreach { format => - val tblName = s"save_$format" - withTable(tblName) { - // create the table (if we can -- not all createAndSave() methods support all formats, - // eg. hive tables don't support json) - if (createAndSave.createAndSave(readWithCorrection, tblName, Some("UTC"), format)) { - // make sure it has the right timezone, and the data is correct. - checkHasTz(spark, tblName, Some("UTC")) - checkTableData(tblName, format) - - // also try to copy this table directly into another table with a different timezone - // setting, for all formats. - ctasFunctions().foreach { ctas => - formats.foreach { destFormat => - val destTableUTC = s"copy_to_utc_$format" - val destTableNoTZ = s"copy_to_no_tz_$format" - withTable(destTableUTC, destTableNoTZ) { - val ctasSupported = ctas.createTableFromSourceTable(source = tblName, - dest = destTableUTC, destTz = Some("UTC"), destFormat = destFormat) - if (ctasSupported) { - checkHasTz(spark, destTableUTC, Some("UTC")) - checkTableData(destTableUTC, destFormat) - - ctas.createTableFromSourceTable(source = tblName, dest = destTableNoTZ, - destTz = None, destFormat = destFormat) - checkHasTz(spark, destTableNoTZ, None) - checkTableData(destTableNoTZ, destFormat) - - // By now, we've checked that the data in both tables is different in terms - // of the raw values on disk, but they are the same after we apply the - // timezone conversions from the table properties. Just to be extra-sure, - // we join the tables and make sure its OK. - val joinedRows = spark.sql( - s"""SELECT a.display, a.ts - |FROM $tblName AS a - |JOIN $destTableUTC AS b - |ON (a.ts = b.ts)""".stripMargin).collect() - assert(joinedRows.size === 4) - joinedRows.foreach { row => - assert(row.getAs[String]("display") === - row.getAs[Timestamp]("ts").toString()) - } - } - } - } - } - - // Finally, try changing the tbl timezone. This destroys integrity - // of the existing data, but at this point we're just checking we can change - // the metadata - spark.sql( - s"""ALTER TABLE $tblName SET TBLPROPERTIES ("$key"="America/Los_Angeles")""") - checkHasTz(spark, tblName, Some("America/Los_Angeles")) - - spark.sql(s"""ALTER TABLE $tblName UNSET TBLPROPERTIES ("$key")""") - checkHasTz(spark, tblName, None) - - spark.sql(s"""ALTER TABLE $tblName SET TBLPROPERTIES ("$key"="UTC")""") - checkHasTz(spark, tblName, Some("UTC")) - } + val tblName = s"save_$format" + withTable(tblName) { + // create the table (if we can -- not all createAndSave() methods support all formats, + // eg. hive tables don't support json) + createFn.createAndSave(readWithCorrection, tblName, Some("UTC")) + // make sure it has the right timezone, and the data is correct. + checkHasTz(spark, tblName, Some("UTC")) + checkTableData(tblName, createFn.format) + + // also try to copy this table directly into another table with a different timezone + // setting, for all formats. + val destTableUTC = s"copy_to_utc_$destFormat" + val destTableNoTZ = s"copy_to_no_tz_$destFormat" + withTable(destTableUTC, destTableNoTZ) { + ctasFn.createFromSource(tblName, destTableUTC, Some("UTC")) + checkHasTz(spark, destTableUTC, Some("UTC")) + checkTableData(destTableUTC, destFormat) + + ctasFn.createFromSource(tblName, destTableNoTZ, None) + checkHasTz(spark, destTableNoTZ, None) + checkTableData(destTableNoTZ, destFormat) + + // By now, we've checked that the data in both tables is different in terms + // of the raw values on disk, but they are the same after we apply the + // timezone conversions from the table properties. Just to be extra-sure, + // we join the tables and make sure its OK. + val joinedRows = spark.sql( + s"""SELECT a.display, a.ts + |FROM $tblName AS a + |JOIN $destTableUTC AS b + |ON (a.ts = b.ts)""".stripMargin).collect() + assert(joinedRows.size === 4) + joinedRows.foreach { row => + assert(row.getAs[String]("display") === + row.getAs[Timestamp]("ts").toString()) } } + + // Finally, try changing the tbl timezone. This destroys integrity + // of the existing data, but at this point we're just checking we can change + // the metadata + spark.sql( + s"""ALTER TABLE $tblName SET TBLPROPERTIES ("$key"="America/Los_Angeles")""") + checkHasTz(spark, tblName, Some("America/Los_Angeles")) + + spark.sql(s"""ALTER TABLE $tblName UNSET TBLPROPERTIES ("$key")""") + checkHasTz(spark, tblName, None) + + spark.sql(s"""ALTER TABLE $tblName SET TBLPROPERTIES ("$key"="UTC")""") + checkHasTz(spark, tblName, Some("UTC")) } } } - test("SPARK-12297: exception on bad timezone") { + test("exception on bad timezone") { // make sure there is an exception anytime we try to read or write with a bad timezone val key = DateTimeUtils.TIMEZONE_PROPERTY val badVal = "Blart Versenwald III" @@ -302,19 +288,19 @@ abstract class BaseAdjustTimestampsSuite extends SparkPlanTest with SQLTestUtils } } - createAndSaveTableFunctions().foreach { createAndSave => + createAndSaveTableFunctions().foreach { case (_, createFn) => hasBadTzException{ - createAndSave.createAndSave(data.toDF(), "bad_tz_table", Some(badVal), "parquet") + createFn.createAndSave(data.toDF(), "bad_tz_table", Some(badVal)) } - createAndSave.createAndSave(data.toDF(), "bad_tz_table", None, "parquet") + createFn.createAndSave(data.toDF(), "bad_tz_table", None) hasBadTzException { spark.sql(s"""ALTER TABLE bad_tz_table SET TBLPROPERTIES("$key"="$badVal")""") } } } - test("SPARK-12297: insertInto must not specify timezone") { + test("insertInto must not specify timezone") { // You can't specify the timezone for just a portion of inserted data. You can only specify // the timezone for the *entire* table (data previously in the table and any future data) so // complain loudly if the user tries to set the timezone on an insert. @@ -329,7 +315,7 @@ abstract class BaseAdjustTimestampsSuite extends SparkPlanTest with SQLTestUtils } } - test("SPARK-12297: refuse table timezone on views") { + test("disallow table timezone on views") { val key = DateTimeUtils.TIMEZONE_PROPERTY val originalData = createRawData(spark) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveAdjustTimestampsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveAdjustTimestampsSuite.scala index 753fef31d179f..79111fbfda026 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveAdjustTimestampsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveAdjustTimestampsSuite.scala @@ -23,26 +23,24 @@ import org.apache.spark.sql.hive.test.TestHiveSingleton class HiveAdjustTimestampsSuite extends BaseAdjustTimestampsSuite with TestHiveSingleton { - override protected def createAndSaveTableFunctions(): Seq[CreateAndSaveTable] = { - super.createAndSaveTableFunctions() ++ - Seq(true, false).map(new CreateHiveTableAndInsert(_)) + override protected def createAndSaveTableFunctions(): Map[String, CreateAndSaveTable] = { + val hiveFns = Map( + "hive_parquet" -> new CreateHiveTableAndInsert(true), + "hive_parquet_no_conversion" -> new CreateHiveTableAndInsert(false)) + + super.createAndSaveTableFunctions() ++ hiveFns } - override protected def ctasFunctions(): Seq[CTAS] = { - super.ctasFunctions() ++ - Seq(true, false).map(new CreateHiveTableWithTimezoneAndInsert(_)) + override protected def ctasFunctions(): Map[String, CTAS] = { + val hiveFns = Map( + "hive_parquet" -> new CreateHiveTableWithTimezoneAndInsert(true), + "hive_parquet_no_conversion" -> new CreateHiveTableWithTimezoneAndInsert(false)) + + super.ctasFunctions() ++ hiveFns } class CreateHiveTableAndInsert(convertMetastore: Boolean) extends CreateAndSaveTable { - override def createAndSave( - df: DataFrame, - table: String, - tzOpt: Option[String], - format: String): Boolean = { - if (format != "parquet") { - return false - } - + override def createAndSave(df: DataFrame, table: String, tzOpt: Option[String]): Unit = { withSQLConf(HiveUtils.CONVERT_METASTORE_PARQUET.key -> convertMetastore.toString) { val tblProperties = tzOpt.map { tz => s"""TBLPROPERTIES ("${DateTimeUtils.TIMEZONE_PROPERTY}"="$tz")""" @@ -52,26 +50,18 @@ class HiveAdjustTimestampsSuite extends BaseAdjustTimestampsSuite with TestHiveS | display string, | ts timestamp |) - |STORED AS $format + |STORED AS parquet |$tblProperties |""".stripMargin) df.write.insertInto(table) } - - true } + + override val format: String = "parquet" } class CreateHiveTableWithTimezoneAndInsert(convertMetastore: Boolean) extends CTAS { - override def createTableFromSourceTable( - source: String, - dest: String, - destTz: Option[String], - destFormat: String): Boolean = { - if (destFormat != "parquet") { - return false - } - + override def createFromSource(source: String, dest: String, destTz: Option[String]): Unit = { withSQLConf(HiveUtils.CONVERT_METASTORE_PARQUET.key -> convertMetastore.toString) { val tblProperties = destTz.map { tz => s"""TBLPROPERTIES ("${DateTimeUtils.TIMEZONE_PROPERTY}"="$tz")""" @@ -82,17 +72,15 @@ class HiveAdjustTimestampsSuite extends BaseAdjustTimestampsSuite with TestHiveS | display string, | ts timestamp |) - |STORED AS $destFormat + |STORED AS parquet |$tblProperties |""".stripMargin) spark.sql(s"insert into $dest select * from $source") } - - true } } - test("SPARK-12297: copy table timezone in CREATE TABLE LIKE") { + test("copy table timezone in CREATE TABLE LIKE") { val key = DateTimeUtils.TIMEZONE_PROPERTY withTable("orig_hive", "copy_hive", "orig_ds", "copy_ds") { spark.sql( From db0d527f56904fc6ad6574ba0457bb5a783c09f4 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 9 Oct 2017 11:57:10 -0700 Subject: [PATCH 18/23] Handle HiveTableRelation as input. Also some other minor fixes and cleanups to the code. --- .../datasources/AdjustTimestamps.scala | 23 +++++++++-- .../datasources/AdjustTimestampsSuite.scala | 17 +++++---- .../spark/sql/hive/HiveStrategies.scala | 2 +- .../sql/hive/HiveAdjustTimestampsSuite.scala | 38 ++++++++++--------- 4 files changed, 52 insertions(+), 28 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/AdjustTimestamps.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/AdjustTimestamps.scala index d8ba4744d70c3..5d9597021e6f8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/AdjustTimestamps.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/AdjustTimestamps.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.datasources import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedException -import org.apache.spark.sql.catalyst.catalog.CatalogTable +import org.apache.spark.sql.catalyst.catalog.{CatalogTable, HiveTableRelation} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule @@ -38,7 +38,8 @@ abstract class BaseAdjustTimestampsRule(sparkSession: SparkSession) extends Rule ): (LogicalPlan, Map[ExprId, NamedExpression]) = plan match { case alreadyConverted@Project(exprs, _) if hasCorrection(exprs) => (alreadyConverted, Map()) - case lr@LogicalRelation(fsRelation: HadoopFsRelation, _, _, _) => + + case lr @ LogicalRelation(fsRelation: HadoopFsRelation, _, _, _) => val tzOpt = extractTableTz(lr.catalogTable, fsRelation.options) tzOpt.flatMap { tableTz => // the table has a timezone set, so after reading the data, apply a conversion @@ -56,6 +57,22 @@ abstract class BaseAdjustTimestampsRule(sparkSession: SparkSession) extends Rule None } }.getOrElse((lr, Map())) + + case relation @ HiveTableRelation(table, cols, parts) => + val tzOpt = extractTableTz(Some(table), Map()) + tzOpt.flatMap { tz => + val toTz = sparkSession.sessionState.conf.sessionLocalTimeZone + if (toTz != tz) { + logDebug(s"table tz = $tz; converting to current session tz = $toTz") + // find timestamp columns, and convert their tz + convertTzForAllTimestamps(relation, tz, toTz).map { case (fields, replacements) => + (new Project(fields, relation), replacements) + } + } else { + None + } + }.getOrElse((relation, Map())) + case other => // first, process all the children -- this ensures we have the right renames in scope. var newReplacements = Map[ExprId, NamedExpression]() @@ -170,7 +187,7 @@ case class AdjustTimestamps(sparkSession: SparkSession) // The query might be reading from a parquet table which requires a different conversion; // this makes sure we apply the correct conversions there. val (fixedQuery, _) = convertInputs(insert.query) - val fixedOutput = writeConversion(insert.catalogTable, insert.options, insert.query) + val fixedOutput = writeConversion(insert.catalogTable, insert.options, fixedQuery) insert.copy(query = fixedOutput) case other => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/AdjustTimestampsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/AdjustTimestampsSuite.scala index e0610e41790fb..f3c184a960054 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/AdjustTimestampsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/AdjustTimestampsSuite.scala @@ -137,9 +137,12 @@ abstract class BaseAdjustTimestampsSuite extends SparkPlanTest with SQLTestUtils * into it. */ def createFromSource(source: String, dest: String, destTz: Option[String]): Unit + + /** The target table's format. */ + val format: String } - class DatasourceCTAS(format: String) extends CTAS { + class DatasourceCTAS(override val format: String) extends CTAS { override def createFromSource(source: String, dest: String, destTz: Option[String]): Unit = { val writer = spark.sql(s"select * from $source").write.format(format) destTz.foreach { writer.option(DateTimeUtils.TIMEZONE_PROPERTY, _)} @@ -177,11 +180,11 @@ abstract class BaseAdjustTimestampsSuite extends SparkPlanTest with SQLTestUtils // These queries should return the entire dataset, but if the predicates were // applied to the raw values in parquet, they would incorrectly filter data out. Seq( - ">" -> "2015-12-31 22:00:00", - "<" -> "2016-01-01 02:00:00" - ).foreach { case (comparison, value) => + "ts > '2015-12-31 22:00:00'", + "ts < '2016-01-01 02:00:00'" + ).foreach { filter => val query = - s"select ts from $table where ts $comparison '$value'" + s"select ts from $table where $filter" val countWithFilter = spark.sql(query).count() assert(countWithFilter === 4, query) } @@ -227,11 +230,11 @@ abstract class BaseAdjustTimestampsSuite extends SparkPlanTest with SQLTestUtils withTable(destTableUTC, destTableNoTZ) { ctasFn.createFromSource(tblName, destTableUTC, Some("UTC")) checkHasTz(spark, destTableUTC, Some("UTC")) - checkTableData(destTableUTC, destFormat) + checkTableData(destTableUTC, ctasFn.format) ctasFn.createFromSource(tblName, destTableNoTZ, None) checkHasTz(spark, destTableNoTZ, None) - checkTableData(destTableNoTZ, destFormat) + checkTableData(destTableNoTZ, ctasFn.format) // By now, we've checked that the data in both tables is different in terms // of the raw values on disk, but they are the same after we apply the diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 11463f61f5347..be76502cb0be4 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -234,7 +234,7 @@ case class HiveAdjustTimestamps(sparkSession: SparkSession) // The query might be reading from a parquet table which requires a different conversion; // this makes sure we apply the correct conversions there. val (fixedQuery, _) = convertInputs(insert.query) - val fixedOutput = writeConversion(Some(insert.table), Map(), insert.query) + val fixedOutput = writeConversion(Some(insert.table), Map(), fixedQuery) insert.copy(query = fixedOutput) case other => diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveAdjustTimestampsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveAdjustTimestampsSuite.scala index 79111fbfda026..9716cbc0517b9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveAdjustTimestampsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveAdjustTimestampsSuite.scala @@ -25,13 +25,17 @@ class HiveAdjustTimestampsSuite extends BaseAdjustTimestampsSuite with TestHiveS override protected def createAndSaveTableFunctions(): Map[String, CreateAndSaveTable] = { val hiveFns = Map( - "hive_parquet" -> new CreateHiveTableAndInsert(true), - "hive_parquet_no_conversion" -> new CreateHiveTableAndInsert(false)) + "hive_parquet" -> new CreateHiveTableAndInsert()) super.createAndSaveTableFunctions() ++ hiveFns } override protected def ctasFunctions(): Map[String, CTAS] = { + // Disabling metastore conversion will also modify how data is read when the the CTAS query is + // run if the source is a Hive table; so, the test that uses "hive_parquet" as the source and + // "hive_parquet_no_conversion" as the target is actually using the "no metastore conversion" + // path for both, making it unnecessary to also have the "no conversion" case in the save + // functions. val hiveFns = Map( "hive_parquet" -> new CreateHiveTableWithTimezoneAndInsert(true), "hive_parquet_no_conversion" -> new CreateHiveTableWithTimezoneAndInsert(false)) @@ -39,22 +43,20 @@ class HiveAdjustTimestampsSuite extends BaseAdjustTimestampsSuite with TestHiveS super.ctasFunctions() ++ hiveFns } - class CreateHiveTableAndInsert(convertMetastore: Boolean) extends CreateAndSaveTable { + class CreateHiveTableAndInsert extends CreateAndSaveTable { override def createAndSave(df: DataFrame, table: String, tzOpt: Option[String]): Unit = { - withSQLConf(HiveUtils.CONVERT_METASTORE_PARQUET.key -> convertMetastore.toString) { - val tblProperties = tzOpt.map { tz => - s"""TBLPROPERTIES ("${DateTimeUtils.TIMEZONE_PROPERTY}"="$tz")""" - }.getOrElse("") - spark.sql( - s"""CREATE TABLE $table ( - | display string, - | ts timestamp - |) - |STORED AS parquet - |$tblProperties - |""".stripMargin) - df.write.insertInto(table) - } + val tblProperties = tzOpt.map { tz => + s"""TBLPROPERTIES ("${DateTimeUtils.TIMEZONE_PROPERTY}"="$tz")""" + }.getOrElse("") + spark.sql( + s"""CREATE TABLE $table ( + | display string, + | ts timestamp + |) + |STORED AS parquet + |$tblProperties + |""".stripMargin) + df.write.insertInto(table) } override val format: String = "parquet" @@ -78,6 +80,8 @@ class HiveAdjustTimestampsSuite extends BaseAdjustTimestampsSuite with TestHiveS spark.sql(s"insert into $dest select * from $source") } } + + override val format: String = "parquet" } test("copy table timezone in CREATE TABLE LIKE") { From e74ce2de90193d07eabf82f905dc5d76384b1d00 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 9 Oct 2017 12:47:19 -0700 Subject: [PATCH 19/23] Add some constants to replace hardcoded values in tests. --- .../datasources/AdjustTimestamps.scala | 1 - .../datasources/AdjustTimestampsSuite.scala | 64 +++++++++---------- .../sql/hive/HiveAdjustTimestampsSuite.scala | 11 ++-- 3 files changed, 36 insertions(+), 40 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/AdjustTimestamps.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/AdjustTimestamps.scala index 5d9597021e6f8..2cf79a4a6a9fd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/AdjustTimestamps.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/AdjustTimestamps.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.execution.datasources import org.apache.spark.sql.{AnalysisException, SparkSession} -import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedException import org.apache.spark.sql.catalyst.catalog.{CatalogTable, HiveTableRelation} import org.apache.spark.sql.catalyst.expressions._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/AdjustTimestampsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/AdjustTimestampsSuite.scala index f3c184a960054..10b1558e74c7b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/AdjustTimestampsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/AdjustTimestampsSuite.scala @@ -32,11 +32,16 @@ import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} abstract class BaseAdjustTimestampsSuite extends SparkPlanTest with SQLTestUtils with BeforeAndAfterAll { + protected val SESSION_TZ = "America/Los_Angeles" + protected val TABLE_TZ = "Europe/Berlin" + protected val UTC = "UTC" + protected val TZ_KEY = DateTimeUtils.TIMEZONE_PROPERTY + var originalTz: TimeZone = _ protected override def beforeAll(): Unit = { super.beforeAll() originalTz = TimeZone.getDefault() - TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) + TimeZone.setDefault(TimeZone.getTimeZone(SESSION_TZ)) } protected override def afterAll(): Unit = { @@ -59,7 +64,7 @@ abstract class BaseAdjustTimestampsSuite extends SparkPlanTest with SQLTestUtils val originalTz = TimeZone.getDefault try { desiredTimestampStrings.flatMap { timestampString => - Seq("America/Los_Angeles", "Europe/Berlin", "UTC").map { tzId => + Seq(SESSION_TZ, TABLE_TZ, UTC).map { tzId => TimeZone.setDefault(TimeZone.getTimeZone(tzId)) val timestamp = Timestamp.valueOf(timestampString) (timestampString, tzId) -> timestamp.getTime() @@ -104,15 +109,11 @@ abstract class BaseAdjustTimestampsSuite extends SparkPlanTest with SQLTestUtils // we want to test that this works w/ hive-only methods as well, so provide a few extension // points so we can also easily re-use this with hive support. protected def createAndSaveTableFunctions(): Map[String, CreateAndSaveTable] = { - formats.map { format => - (format, new CreateAndSaveDatasourceTable(format)) - }.toMap + formats.map { f => (f, new CreateAndSaveDatasourceTable(f)) }.toMap } protected def ctasFunctions(): Map[String, CTAS] = { - formats.map { format => - (format, new DatasourceCTAS(format)) - }.toMap + formats.map { f => (f, new DatasourceCTAS(f)) }.toMap } trait CreateAndSaveTable { @@ -163,13 +164,12 @@ abstract class BaseAdjustTimestampsSuite extends SparkPlanTest with SQLTestUtils destFormat: String, createFn: CreateAndSaveTable, ctasFn: CTAS): Unit = { - assert(TimeZone.getDefault.getID() === "America/Los_Angeles") - val key = DateTimeUtils.TIMEZONE_PROPERTY + assert(TimeZone.getDefault.getID() === SESSION_TZ) val originalData = createRawData(spark) withTempPath { basePath => val dsPath = new File(basePath, "dsFlat").getAbsolutePath originalData.write - .option(key, "Europe/Berlin") + .option(TZ_KEY, TABLE_TZ) .parquet(dsPath) /** @@ -186,26 +186,26 @@ abstract class BaseAdjustTimestampsSuite extends SparkPlanTest with SQLTestUtils val query = s"select ts from $table where $filter" val countWithFilter = spark.sql(query).count() - assert(countWithFilter === 4, query) + assert(countWithFilter === desiredTimestampStrings.size, query) } - // also, read the raw parquet data, without any TZ correction, and make sure the raw + // also, read the raw table data, without any TZ correction, and make sure the raw // values have been adjusted as we expect. val tableMeta = spark.sessionState.catalog.getTableMetadata(TableIdentifier(table)) val location = tableMeta.location.toString() val tz = tableMeta.properties.get(DateTimeUtils.TIMEZONE_PROPERTY) // some formats need the schema specified val df = spark.read.schema(originalData.schema).format(format).load(location) - checkRawData(df, tz.getOrElse("America/Los_Angeles")) + checkRawData(df, tz.getOrElse(SESSION_TZ)) } // read it back, without supplying the right timezone. Won't match the original, but we // expect specific values. val readNoCorrection = spark.read.parquet(dsPath) - checkRawData(readNoCorrection, "Europe/Berlin") + checkRawData(readNoCorrection, TABLE_TZ) // now read it back *with* the right timezone -- everything should match. - val readWithCorrection = spark.read.option(key, "Europe/Berlin").parquet(dsPath) + val readWithCorrection = spark.read.option(TZ_KEY, TABLE_TZ).parquet(dsPath) readWithCorrection.collect().foreach { row => assert(row.getAs[String]("display") === row.getAs[Timestamp]("ts").toString()) @@ -218,9 +218,9 @@ abstract class BaseAdjustTimestampsSuite extends SparkPlanTest with SQLTestUtils withTable(tblName) { // create the table (if we can -- not all createAndSave() methods support all formats, // eg. hive tables don't support json) - createFn.createAndSave(readWithCorrection, tblName, Some("UTC")) + createFn.createAndSave(readWithCorrection, tblName, Some(UTC)) // make sure it has the right timezone, and the data is correct. - checkHasTz(spark, tblName, Some("UTC")) + checkHasTz(spark, tblName, Some(UTC)) checkTableData(tblName, createFn.format) // also try to copy this table directly into another table with a different timezone @@ -228,8 +228,8 @@ abstract class BaseAdjustTimestampsSuite extends SparkPlanTest with SQLTestUtils val destTableUTC = s"copy_to_utc_$destFormat" val destTableNoTZ = s"copy_to_no_tz_$destFormat" withTable(destTableUTC, destTableNoTZ) { - ctasFn.createFromSource(tblName, destTableUTC, Some("UTC")) - checkHasTz(spark, destTableUTC, Some("UTC")) + ctasFn.createFromSource(tblName, destTableUTC, Some(UTC)) + checkHasTz(spark, destTableUTC, Some(UTC)) checkTableData(destTableUTC, ctasFn.format) ctasFn.createFromSource(tblName, destTableNoTZ, None) @@ -256,21 +256,20 @@ abstract class BaseAdjustTimestampsSuite extends SparkPlanTest with SQLTestUtils // of the existing data, but at this point we're just checking we can change // the metadata spark.sql( - s"""ALTER TABLE $tblName SET TBLPROPERTIES ("$key"="America/Los_Angeles")""") - checkHasTz(spark, tblName, Some("America/Los_Angeles")) + s"""ALTER TABLE $tblName SET TBLPROPERTIES ("$TZ_KEY"="$SESSION_TZ")""") + checkHasTz(spark, tblName, Some(SESSION_TZ)) - spark.sql(s"""ALTER TABLE $tblName UNSET TBLPROPERTIES ("$key")""") + spark.sql(s"""ALTER TABLE $tblName UNSET TBLPROPERTIES ("$TZ_KEY")""") checkHasTz(spark, tblName, None) - spark.sql(s"""ALTER TABLE $tblName SET TBLPROPERTIES ("$key"="UTC")""") - checkHasTz(spark, tblName, Some("UTC")) + spark.sql(s"""ALTER TABLE $tblName SET TBLPROPERTIES ("$TZ_KEY"="$UTC")""") + checkHasTz(spark, tblName, Some(UTC)) } } } test("exception on bad timezone") { // make sure there is an exception anytime we try to read or write with a bad timezone - val key = DateTimeUtils.TIMEZONE_PROPERTY val badVal = "Blart Versenwald III" val data = createRawData(spark) def hasBadTzException(command: => Unit): Unit = { @@ -282,12 +281,12 @@ abstract class BaseAdjustTimestampsSuite extends SparkPlanTest with SQLTestUtils withTempPath { p => hasBadTzException { - data.write.option(key, badVal).parquet(p.getAbsolutePath) + data.write.option(TZ_KEY, badVal).parquet(p.getAbsolutePath) } data.write.parquet(p.getAbsolutePath) hasBadTzException { - spark.read.option(key, badVal).parquet(p.getAbsolutePath) + spark.read.option(TZ_KEY, badVal).parquet(p.getAbsolutePath) } } @@ -298,7 +297,7 @@ abstract class BaseAdjustTimestampsSuite extends SparkPlanTest with SQLTestUtils createFn.createAndSave(data.toDF(), "bad_tz_table", None) hasBadTzException { - spark.sql(s"""ALTER TABLE bad_tz_table SET TBLPROPERTIES("$key"="$badVal")""") + spark.sql(s"""ALTER TABLE bad_tz_table SET TBLPROPERTIES("$TZ_KEY"="$badVal")""") } } } @@ -311,7 +310,7 @@ abstract class BaseAdjustTimestampsSuite extends SparkPlanTest with SQLTestUtils val origData = createRawData(spark) origData.write.saveAsTable("some_table") val exc = intercept[AnalysisException]{ - createRawData(spark).write.option(DateTimeUtils.TIMEZONE_PROPERTY, "UTC") + createRawData(spark).write.option(DateTimeUtils.TIMEZONE_PROPERTY, UTC) .insertInto("some_table") } assert(exc.getMessage.contains("Cannot provide a table timezone on insert")) @@ -319,7 +318,6 @@ abstract class BaseAdjustTimestampsSuite extends SparkPlanTest with SQLTestUtils } test("disallow table timezone on views") { - val key = DateTimeUtils.TIMEZONE_PROPERTY val originalData = createRawData(spark) withTable("ok_table") { @@ -327,14 +325,14 @@ abstract class BaseAdjustTimestampsSuite extends SparkPlanTest with SQLTestUtils withView("view_with_tz") { val exc1 = intercept[AnalysisException]{ spark.sql(s"""CREATE VIEW view_with_tz - |TBLPROPERTIES ("$key"="UTC") + |TBLPROPERTIES ("$TZ_KEY"="$UTC") |AS SELECT * FROM ok_table """.stripMargin) } assert(exc1.getMessage.contains("Timezone cannot be set for view")) spark.sql("CREATE VIEW view_with_tz AS SELECT * FROM ok_table") val exc2 = intercept[AnalysisException]{ - spark.sql(s"""ALTER VIEW view_with_tz SET TBLPROPERTIES("$key"="UTC")""") + spark.sql(s"""ALTER VIEW view_with_tz SET TBLPROPERTIES("$TZ_KEY"="$UTC")""") } assert(exc2.getMessage.contains("Timezone cannot be set for view")) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveAdjustTimestampsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveAdjustTimestampsSuite.scala index 9716cbc0517b9..250f4d6801b56 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveAdjustTimestampsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveAdjustTimestampsSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.hive import org.apache.spark.sql.DataFrame import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.execution.datasources.{AdjustTimestamps, BaseAdjustTimestampsSuite} +import org.apache.spark.sql.execution.datasources.BaseAdjustTimestampsSuite import org.apache.spark.sql.hive.test.TestHiveSingleton class HiveAdjustTimestampsSuite extends BaseAdjustTimestampsSuite with TestHiveSingleton { @@ -85,7 +85,6 @@ class HiveAdjustTimestampsSuite extends BaseAdjustTimestampsSuite with TestHiveS } test("copy table timezone in CREATE TABLE LIKE") { - val key = DateTimeUtils.TIMEZONE_PROPERTY withTable("orig_hive", "copy_hive", "orig_ds", "copy_ds") { spark.sql( s"""CREATE TABLE orig_hive ( @@ -93,15 +92,15 @@ class HiveAdjustTimestampsSuite extends BaseAdjustTimestampsSuite with TestHiveS | ts timestamp |) |STORED AS parquet - |TBLPROPERTIES ("$key"="UTC") + |TBLPROPERTIES ("$TZ_KEY"="$UTC") |""". stripMargin) spark.sql("CREATE TABLE copy_hive LIKE orig_hive") - checkHasTz(spark, "copy_hive", Some("UTC")) + checkHasTz(spark, "copy_hive", Some(UTC)) - createRawData(spark).write.option(key, "America/New_York").saveAsTable("orig_ds") + createRawData(spark).write.option(TZ_KEY, TABLE_TZ).saveAsTable("orig_ds") spark.sql("CREATE TABLE copy_ds LIKE orig_ds") - checkHasTz(spark, "copy_ds", Some("America/New_York")) + checkHasTz(spark, "copy_ds", Some(TABLE_TZ)) } } From 1eaa0452594e0e94808c8af1ca1e83ed3880621b Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 9 Oct 2017 13:54:17 -0700 Subject: [PATCH 20/23] Minor nit. --- .../spark/sql/execution/datasources/AdjustTimestamps.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/AdjustTimestamps.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/AdjustTimestamps.scala index 2cf79a4a6a9fd..9ff5039c71f84 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/AdjustTimestamps.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/AdjustTimestamps.scala @@ -57,7 +57,7 @@ abstract class BaseAdjustTimestampsRule(sparkSession: SparkSession) extends Rule } }.getOrElse((lr, Map())) - case relation @ HiveTableRelation(table, cols, parts) => + case relation @ HiveTableRelation(table, _, _) => val tzOpt = extractTableTz(Some(table), Map()) tzOpt.flatMap { tz => val toTz = sparkSession.sessionState.conf.sessionLocalTimeZone From 5c03e07ca0c3ec95b580280b7efe7aa7e1b5d734 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 10 Oct 2017 09:42:51 -0700 Subject: [PATCH 21/23] Cleanup AdjustTimestamps. Running the rule during resolution also allowed to do all the needed ajustments with a single rule (instead of needing a Hive-specific rule for InsertIntoHiveTable). --- .../datasources/AdjustTimestamps.scala | 236 ++++++------------ .../internal/BaseSessionStateBuilder.scala | 2 +- .../sql/hive/HiveSessionStateBuilder.scala | 3 +- .../spark/sql/hive/HiveStrategies.scala | 29 +-- 4 files changed, 86 insertions(+), 184 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/AdjustTimestamps.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/AdjustTimestamps.scala index 9ff5039c71f84..e6a1680d5db5f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/AdjustTimestamps.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/AdjustTimestamps.scala @@ -16,183 +16,113 @@ */ package org.apache.spark.sql.execution.datasources -import org.apache.spark.sql.{AnalysisException, SparkSession} -import org.apache.spark.sql.catalyst.analysis.UnresolvedException +import org.apache.spark.sql.{AnalysisException} import org.apache.spark.sql.catalyst.catalog.{CatalogTable, HiveTableRelation} import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} +import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{StringType, TimestampType} -abstract class BaseAdjustTimestampsRule(sparkSession: SparkSession) extends Rule[LogicalPlan] { +/** + * Apply a correction to data loaded from, or saved to, tables that have a configured time zone, so + * that timestamps can be read like TIMESTAMP WITHOUT TIMEZONE. This gives correct behavior if you + * process data with machines in different timezones, or if you access the data from multiple SQL + * engines. + */ +case class AdjustTimestamps(conf: SQLConf) extends Rule[LogicalPlan] { - /** - * Apply the correction to all timestamp inputs, and replace all references to the raw attributes - * with the new converted inputs. - * @return The converted plan, and the replacements to be applied further up the plan - */ - protected def convertInputs( - plan: LogicalPlan - ): (LogicalPlan, Map[ExprId, NamedExpression]) = plan match { - case alreadyConverted@Project(exprs, _) if hasCorrection(exprs) => - (alreadyConverted, Map()) + def apply(plan: LogicalPlan): LogicalPlan = plan match { + case insert: InsertIntoHadoopFsRelationCommand => + val adjusted = adjustTimestampsForWrite(insert.query, insert.catalogTable, insert.options) + insert.copy(query = adjusted) - case lr @ LogicalRelation(fsRelation: HadoopFsRelation, _, _, _) => - val tzOpt = extractTableTz(lr.catalogTable, fsRelation.options) - tzOpt.flatMap { tableTz => - // the table has a timezone set, so after reading the data, apply a conversion - - // SessionTZ (instead of JVM TZ) will make the time display correctly in SQL queries, but - // incorrectly if you pull Timestamp objects out (eg. with a dataset.collect()) - val toTz = sparkSession.sessionState.conf.sessionLocalTimeZone - if (toTz != tableTz) { - logDebug(s"table tz = $tableTz; converting to current session tz = $toTz") - // find timestamp columns, and convert their tz - convertTzForAllTimestamps(lr, tableTz, toTz).map { case (fields, replacements) => - (new Project(fields, lr), replacements) - } - } else { - None - } - }.getOrElse((lr, Map())) - - case relation @ HiveTableRelation(table, _, _) => - val tzOpt = extractTableTz(Some(table), Map()) - tzOpt.flatMap { tz => - val toTz = sparkSession.sessionState.conf.sessionLocalTimeZone - if (toTz != tz) { - logDebug(s"table tz = $tz; converting to current session tz = $toTz") - // find timestamp columns, and convert their tz - convertTzForAllTimestamps(relation, tz, toTz).map { case (fields, replacements) => - (new Project(fields, relation), replacements) - } - } else { - None - } - }.getOrElse((relation, Map())) + case insert @ InsertIntoTable(table: HiveTableRelation, _, query, _, _) => + val adjusted = adjustTimestampsForWrite(insert.query, Some(table.tableMeta), Map()) + insert.copy(query = adjusted) case other => - // first, process all the children -- this ensures we have the right renames in scope. - var newReplacements = Map[ExprId, NamedExpression]() - val fixedPlan = other.mapChildren { originalPlan => - val (newPlan, extraReplacements) = convertInputs(originalPlan) - newReplacements ++= extraReplacements - newPlan - } - // now we need to adjust all names to use the new version. - val fixedExpressions = fixedPlan.mapExpressions { outerExp => - val adjustedExp = outerExp.transformUp { case exp: NamedExpression => - try { - newReplacements.get(exp.exprId).getOrElse(exp) - } catch { - // UnresolvedAttributes etc. will cause problems later anyway, we just dont' want to - // expose the error here - case ue: UnresolvedException[_] => exp - } - } - adjustedExp - } - (fixedExpressions, newReplacements) + convertInputs(plan) } - protected def hasCorrection(exprs: Seq[Expression]): Boolean = { - exprs.exists { expr => - expr.isInstanceOf[TimestampTimezoneCorrection] || hasCorrection(expr.children) - } + private def convertInputs(plan: LogicalPlan): LogicalPlan = plan match { + case adjusted @ Project(exprs, _) if hasCorrection(exprs) => + adjusted + + case lr @ LogicalRelation(fsRelation: HadoopFsRelation, _, _, _) => + adjustTimestamps(lr, lr.catalogTable, fsRelation.options, true) + + case hr @ HiveTableRelation(table, _, _) => + adjustTimestamps(hr, Some(table), Map(), true) + + case other => + other.mapChildren { originalPlan => + convertInputs(originalPlan) + } } - protected def writeConversion( + private def adjustTimestamps( + plan: LogicalPlan, table: Option[CatalogTable], options: Map[String, String], - query: LogicalPlan): LogicalPlan = { - val tableTz = extractTableTz(table, options) - val internalTz = sparkSession.sessionState.conf.sessionLocalTimeZone - if (tableTz.isDefined && tableTz != internalTz) { - convertTzForAllTimestamps(query, internalTz, tableTz.get).map { case (fields, _) => - new Project(fields, query) - }.getOrElse(query) - } else { - query - } - } - - protected def extractTableTz(options: Map[String, String]): Option[String] = { - options.get(DateTimeUtils.TIMEZONE_PROPERTY) + reading: Boolean): LogicalPlan = { + val tableTz = table.flatMap(_.properties.get(DateTimeUtils.TIMEZONE_PROPERTY)) + .orElse(options.get(DateTimeUtils.TIMEZONE_PROPERTY)) + + tableTz.map { tz => + val sessionTz = conf.sessionLocalTimeZone + val toTz = if (reading) sessionTz else tz + val fromTz = if (reading) tz else sessionTz + logDebug( + s"table tz = $tz; converting ${if (reading) "to" else "from"} session tz = $sessionTz\n") + + var hasTimestamp = false + val adjusted = plan.expressions.map { + case e: NamedExpression if e.dataType == TimestampType => + val adjustment = TimestampTimezoneCorrection(e.toAttribute, + Literal.create(fromTz, StringType), Literal.create(toTz, StringType)) + hasTimestamp = true + Alias(adjustment, e.name)() + + case other: NamedExpression => + other + + case unnamed => + throw new AnalysisException(s"Unexpected expr: $unnamed") + }.toList + + if (hasTimestamp) Project(adjusted, plan) else plan + }.getOrElse(plan) } - protected def extractTableTz( + private def adjustTimestampsForWrite( + query: LogicalPlan, table: Option[CatalogTable], - options: Map[String, String]): Option[String] = { - table.flatMap { tbl => extractTableTz(tbl.properties) }.orElse(extractTableTz(options)) + options: Map[String, String]): LogicalPlan = query match { + case unadjusted if !hasOutputCorrection(unadjusted.expressions) => + // The query might be reading from a table with a configured time zone; this makes sure we + // apply the correct conversions for that data. + val fixedInputs = convertInputs(unadjusted) + adjustTimestamps(fixedInputs, table, options, false) + + case _ => + query } - /** - * Find all timestamp fields in the given relation. For each one, replace it with an expression - * that converts the timezone of the timestamp, and assigns an alias to that new expression. - * (Leave non-timestamp fields alone.) Also return a map from the original id for the timestamp - * field, to the new alias of the timezone-corrected expression. - */ - protected def convertTzForAllTimestamps( - relation: LogicalPlan, - fromTz: String, - toTz: String): Option[(Seq[NamedExpression], Map[ExprId, NamedExpression])] = { - val schema = relation.schema - var foundTs = false - var replacements = Map[ExprId, NamedExpression]() - val modifiedFields: Seq[NamedExpression] = schema.map { field => - val exp = relation.resolve(Seq(field.name), sparkSession.sessionState.conf.resolver) - .getOrElse { - val inputColumns = schema.map(_.name).mkString(", ") - throw new AnalysisException( - s"cannot resolve '${field.name}' given input columns: [$inputColumns]") - } - if (field.dataType == TimestampType) { - foundTs = true - val adjustedTs = Alias( - TimestampTimezoneCorrection( - exp, - Literal.create(fromTz, StringType), - Literal.create(toTz, StringType) - ), - field.name - )() - // we also need to rename all occurrences of this field further up in the plan - // to refer to our new adjusted timestamp, so we pass this replacement up the call stack. - replacements += exp.exprId -> adjustedTs.toAttribute - adjustedTs - } else { - exp - } + private def hasCorrection(exprs: Seq[Expression]): Boolean = { + exprs.exists { expr => + expr.isInstanceOf[TimestampTimezoneCorrection] || hasCorrection(expr.children) } - if (foundTs) Some((modifiedFields, replacements)) else None } -} -/** - * Apply a correction to data loaded from, or saved to, tables that have a configured time zone, so - * that timestamps can be read like TIMESTAMP WITHOUT TIMEZONE. This gives correct behavior if you - * process data with machines in different timezones, or if you access the data from multiple SQL - * engines. - */ -case class AdjustTimestamps(sparkSession: SparkSession) - extends BaseAdjustTimestampsRule(sparkSession) { - - def apply(plan: LogicalPlan): LogicalPlan = { - // we can't use transformUp because we want to terminate recursion if there was already - // timestamp correction, to keep this idempotent. - plan match { - case insert: InsertIntoHadoopFsRelationCommand => - // The query might be reading from a parquet table which requires a different conversion; - // this makes sure we apply the correct conversions there. - val (fixedQuery, _) = convertInputs(insert.query) - val fixedOutput = writeConversion(insert.catalogTable, insert.options, fixedQuery) - insert.copy(query = fixedOutput) - - case other => - // recurse into children to see if we're reading data that needs conversion - val (convertedPlan, _) = convertInputs(plan) - convertedPlan + private def hasOutputCorrection(exprs: Seq[Expression]): Boolean = { + // Output correction is any TimestampTimezoneCorrection that converts from the current + // session's time zone. + val sessionTz = conf.sessionLocalTimeZone + exprs.exists { + case TimestampTimezoneCorrection(_, from, _) => from.toString() == sessionTz + case other => hasOutputCorrection(other.children) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala index d7b027c49292a..beb8db497928e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala @@ -158,13 +158,13 @@ abstract class BaseSessionStateBuilder( override val extendedResolutionRules: Seq[Rule[LogicalPlan]] = new FindDataSourceTable(session) +: new ResolveSQLOnFile(session) +: + AdjustTimestamps(conf) +: customResolutionRules override val postHocResolutionRules: Seq[Rule[LogicalPlan]] = PreprocessTableCreation(session) +: PreprocessTableInsertion(conf) +: DataSourceAnalysis(conf) +: - AdjustTimestamps(session) +: customPostHocResolutionRules override val extendedCheckRules: Seq[LogicalPlan => Unit] = diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala index 2a02375dc9e85..218926385c650 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala @@ -71,6 +71,7 @@ class HiveSessionStateBuilder(session: SparkSession, parentState: Option[Session new ResolveHiveSerdeTable(session) +: new FindDataSourceTable(session) +: new ResolveSQLOnFile(session) +: + AdjustTimestamps(conf) +: customResolutionRules override val postHocResolutionRules: Seq[Rule[LogicalPlan]] = @@ -80,8 +81,6 @@ class HiveSessionStateBuilder(session: SparkSession, parentState: Option[Session PreprocessTableInsertion(conf) +: DataSourceAnalysis(conf) +: HiveAnalysis +: - HiveAdjustTimestamps(session) +: - AdjustTimestamps(session) +: customPostHocResolutionRules override val extendedCheckRules: Seq[LogicalPlan => Unit] = diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index be76502cb0be4..805b3171cdaab 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoDir, InsertIntoTab import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.command.{CreateTableCommand, DDLUtils} -import org.apache.spark.sql.execution.datasources.{BaseAdjustTimestampsRule, CreateTable, LogicalRelation} +import org.apache.spark.sql.execution.datasources.{CreateTable, LogicalRelation} import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, ParquetOptions} import org.apache.spark.sql.hive.execution._ import org.apache.spark.sql.hive.orc.OrcFileFormat @@ -217,33 +217,6 @@ case class RelationConversions( } } -/** - * Apply a correction to data loaded from, or saved to, tables that have a configured time zone, so - * that timestamps can be read like TIMESTAMP WITHOUT TIMEZONE. This gives correct behavior if you - * process data with machines in different timezones, or if you access the data from multiple SQL - * engines. - */ -case class HiveAdjustTimestamps(sparkSession: SparkSession) - extends BaseAdjustTimestampsRule(sparkSession) { - - def apply(plan: LogicalPlan): LogicalPlan = { - // we can't use transformUp because we want to terminate recursion if there was already - // timestamp correction, to keep this idempotent. - plan match { - case insert: InsertIntoHiveTable => - // The query might be reading from a parquet table which requires a different conversion; - // this makes sure we apply the correct conversions there. - val (fixedQuery, _) = convertInputs(insert.query) - val fixedOutput = writeConversion(Some(insert.table), Map(), fixedQuery) - insert.copy(query = fixedOutput) - - case other => - plan - } - } - -} - private[hive] trait HiveStrategies { // Possibly being too clever with types here... or not clever enough. self: SparkPlanner => From 5607160afaf0f5ecd93fa59b97549bec937991b4 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 10 Oct 2017 10:54:26 -0700 Subject: [PATCH 22/23] Small fix. --- .../spark/sql/execution/datasources/AdjustTimestamps.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/AdjustTimestamps.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/AdjustTimestamps.scala index e6a1680d5db5f..2e57c5ea7dc21 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/AdjustTimestamps.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/AdjustTimestamps.scala @@ -86,7 +86,7 @@ case class AdjustTimestamps(conf: SQLConf) extends Rule[LogicalPlan] { Alias(adjustment, e.name)() case other: NamedExpression => - other + other.toAttribute case unnamed => throw new AnalysisException(s"Unexpected expr: $unnamed") From 7e444868f9dd0ca601c806abbfca7ef219f686d9 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 10 Oct 2017 16:29:52 -0700 Subject: [PATCH 23/23] Use constant in more places. --- .../execution/datasources/AdjustTimestampsSuite.scala | 10 +++++----- .../spark/sql/hive/HiveAdjustTimestampsSuite.scala | 5 ++--- 2 files changed, 7 insertions(+), 8 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/AdjustTimestampsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/AdjustTimestampsSuite.scala index 10b1558e74c7b..8a5b7452c8a93 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/AdjustTimestampsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/AdjustTimestampsSuite.scala @@ -84,7 +84,7 @@ abstract class BaseAdjustTimestampsSuite extends SparkPlanTest with SQLTestUtils protected def checkHasTz(spark: SparkSession, table: String, tz: Option[String]): Unit = { val tableMetadata = spark.sessionState.catalog.getTableMetadata(TableIdentifier(table)) - assert(tableMetadata.properties.get(DateTimeUtils.TIMEZONE_PROPERTY) === tz, + assert(tableMetadata.properties.get(TZ_KEY) === tz, s"for table $table") } @@ -127,7 +127,7 @@ abstract class BaseAdjustTimestampsSuite extends SparkPlanTest with SQLTestUtils class CreateAndSaveDatasourceTable(override val format: String) extends CreateAndSaveTable { override def createAndSave(df: DataFrame, table: String, tz: Option[String]): Unit = { val writer = df.write.format(format) - tz.foreach(writer.option(DateTimeUtils.TIMEZONE_PROPERTY, _)) + tz.foreach(writer.option(TZ_KEY, _)) writer.saveAsTable(table) } } @@ -146,7 +146,7 @@ abstract class BaseAdjustTimestampsSuite extends SparkPlanTest with SQLTestUtils class DatasourceCTAS(override val format: String) extends CTAS { override def createFromSource(source: String, dest: String, destTz: Option[String]): Unit = { val writer = spark.sql(s"select * from $source").write.format(format) - destTz.foreach { writer.option(DateTimeUtils.TIMEZONE_PROPERTY, _)} + destTz.foreach { writer.option(TZ_KEY, _)} writer.saveAsTable(dest) } } @@ -193,7 +193,7 @@ abstract class BaseAdjustTimestampsSuite extends SparkPlanTest with SQLTestUtils // values have been adjusted as we expect. val tableMeta = spark.sessionState.catalog.getTableMetadata(TableIdentifier(table)) val location = tableMeta.location.toString() - val tz = tableMeta.properties.get(DateTimeUtils.TIMEZONE_PROPERTY) + val tz = tableMeta.properties.get(TZ_KEY) // some formats need the schema specified val df = spark.read.schema(originalData.schema).format(format).load(location) checkRawData(df, tz.getOrElse(SESSION_TZ)) @@ -310,7 +310,7 @@ abstract class BaseAdjustTimestampsSuite extends SparkPlanTest with SQLTestUtils val origData = createRawData(spark) origData.write.saveAsTable("some_table") val exc = intercept[AnalysisException]{ - createRawData(spark).write.option(DateTimeUtils.TIMEZONE_PROPERTY, UTC) + createRawData(spark).write.option(TZ_KEY, UTC) .insertInto("some_table") } assert(exc.getMessage.contains("Cannot provide a table timezone on insert")) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveAdjustTimestampsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveAdjustTimestampsSuite.scala index 250f4d6801b56..2fd35eb01b886 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveAdjustTimestampsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveAdjustTimestampsSuite.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.hive import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.datasources.BaseAdjustTimestampsSuite import org.apache.spark.sql.hive.test.TestHiveSingleton @@ -46,7 +45,7 @@ class HiveAdjustTimestampsSuite extends BaseAdjustTimestampsSuite with TestHiveS class CreateHiveTableAndInsert extends CreateAndSaveTable { override def createAndSave(df: DataFrame, table: String, tzOpt: Option[String]): Unit = { val tblProperties = tzOpt.map { tz => - s"""TBLPROPERTIES ("${DateTimeUtils.TIMEZONE_PROPERTY}"="$tz")""" + s"""TBLPROPERTIES ("$TZ_KEY"="$tz")""" }.getOrElse("") spark.sql( s"""CREATE TABLE $table ( @@ -66,7 +65,7 @@ class HiveAdjustTimestampsSuite extends BaseAdjustTimestampsSuite with TestHiveS override def createFromSource(source: String, dest: String, destTz: Option[String]): Unit = { withSQLConf(HiveUtils.CONVERT_METASTORE_PARQUET.key -> convertMetastore.toString) { val tblProperties = destTz.map { tz => - s"""TBLPROPERTIES ("${DateTimeUtils.TIMEZONE_PROPERTY}"="$tz")""" + s"""TBLPROPERTIES ("$TZ_KEY"="$tz")""" }.getOrElse("") // this isn't just a "ctas" sql statement b/c that doesn't let us specify the table tz spark.sql(