From 53d0744f8cdb9404bfe84f1e0154606d3442639c Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 26 Jan 2017 20:37:27 -0600 Subject: [PATCH 01/40] very basic test for adjusting read parquet data --- .../parquet/VectorizedColumnReader.java | 6 ++- .../parquet/ParquetFileFormat.scala | 2 + .../parquet/ParquetRowConverter.scala | 39 +++++++++++++------ .../datasources/parquet/ParquetIOSuite.scala | 28 +++++++++++++ 4 files changed, 61 insertions(+), 14 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java index cb51cb499eed..88716e857767 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java @@ -276,7 +276,8 @@ private void decodeDictionaryIds(int rowId, int num, ColumnVector column, // TODO: Convert dictionary of Binaries to dictionary of Longs if (!column.isNullAt(i)) { Binary v = dictionary.decodeToBinary(dictionaryIds.getDictId(i)); - column.putLong(i, ParquetRowConverter.binaryToSQLTimestamp(v)); + column.putLong(i, + ParquetRowConverter.binaryToSQLTimestamp(v, ParquetRowConverter.tz())); } } } else { @@ -401,7 +402,8 @@ private void readBinaryBatch(int rowId, int num, ColumnVector column) throws IOE if (defColumn.readInteger() == maxDefLevel) { column.putLong(rowId + i, // Read 12 bytes for INT96 - ParquetRowConverter.binaryToSQLTimestamp(data.readBinary(12))); + ParquetRowConverter.binaryToSQLTimestamp(data.readBinary(12), + ParquetRowConverter.tz())); } else { column.putNull(rowId + i); } 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 d9831c5d5faf..8a5aeff51bcf 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 @@ -707,4 +707,6 @@ object ParquetFileFormat extends Logging { Failure(cause) }.toOption } + + val PARQUET_TIMEZONE_TABLE_PROPERTY = "parquet.mr.int96.write.zone" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala index 33dcf2f3fd16..c2dfdeb7c600 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala @@ -19,6 +19,10 @@ package org.apache.spark.sql.execution.datasources.parquet import java.math.{BigDecimal, BigInteger} import java.nio.ByteOrder +import java.sql.Timestamp +import java.util.{Calendar, Date, TimeZone} +import java.util.Formatter.DateTime +import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer @@ -257,15 +261,10 @@ private[parquet] class ParquetRowConverter( new ParquetPrimitiveConverter(updater) { // Converts nanosecond timestamps stored as INT96 override def addBinary(value: Binary): Unit = { - assert( - value.length() == 12, - "Timestamps (with nanoseconds) are expected to be stored in 12-byte long binaries, " + - s"but got a ${value.length()}-byte binary.") - - val buf = value.toByteBuffer.order(ByteOrder.LITTLE_ENDIAN) - val timeOfDayNanos = buf.getLong - val julianDay = buf.getInt - updater.setLong(DateTimeUtils.fromJulianDay(julianDay, timeOfDayNanos)) + // TODO timezone from table props + val timestamp = + ParquetRowConverter.binaryToSQLTimestamp(value, ParquetRowConverter.tz) + updater.setLong(timestamp) } } @@ -644,7 +643,14 @@ private[parquet] class ParquetRowConverter( } private[parquet] object ParquetRowConverter { - def binaryToUnscaledLong(binary: Binary): Long = { + + // just for testing + var tz: TimeZone = DateTimeUtils.TimeZoneGMT + def setTimezone(tz: TimeZone): Unit = { + this.tz = tz + } + + def binaryToUnscaledLong(binary: Binary): Long = { // The underlying `ByteBuffer` implementation is guaranteed to be `HeapByteBuffer`, so here // we are using `Binary.toByteBuffer.array()` to steal the underlying byte array without // copying it. @@ -666,12 +672,21 @@ private[parquet] object ParquetRowConverter { unscaled } - def binaryToSQLTimestamp(binary: Binary): SQLTimestamp = { + def binaryToSQLTimestamp(binary: Binary, tz: TimeZone): SQLTimestamp = { assert(binary.length() == 12, s"Timestamps (with nanoseconds) are expected to be stored in" + s" 12-byte long binaries. Found a ${binary.length()}-byte binary instead.") val buffer = binary.toByteBuffer.order(ByteOrder.LITTLE_ENDIAN) val timeOfDayNanos = buffer.getLong val julianDay = buffer.getInt - DateTimeUtils.fromJulianDay(julianDay, timeOfDayNanos) + val utcEpochMicros = DateTimeUtils.fromJulianDay(julianDay, timeOfDayNanos) + // avoid expensive time logic if possible + if (tz != DateTimeUtils.TimeZoneGMT) { + // TODO not really sure what the desired behavior here is ... + val millis = utcEpochMicros / 1000 + val offset = tz.getOffset(millis) + ((millis + offset) * 1000) + (utcEpochMicros % 1000) + } else { + utcEpochMicros + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala index dbdcd230a4de..b174f078141e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.datasources.parquet +import java.util.TimeZone + import scala.collection.JavaConverters._ import scala.collection.mutable import scala.reflect.ClassTag @@ -753,6 +755,32 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { assert(option.compressionCodecClassName == "UNCOMPRESSED") } } + + test("SPARK-12297: Parquet Timestamp & Hive Timezones") { + // Test that we can correctly adjust parquet timestamps for Hive timezone bug. + withTempPath { dir => + // First, lets generate some parquet data we can use to test this + val schema = StructType(StructField("timestamp", TimestampType) :: Nil) + // intentionally pick a few times right around new years, so time zone will effect many fields + val data = spark.sparkContext.parallelize(Seq( + "2015-12-31 23:50:59.123", + "2015-12-31 22:49:59.123", + "2016-01-01 00:39:59.123", + "2016-01-01 01:29:59.123" + ).map { x => Row(java.sql.Timestamp.valueOf(x)) }) + // TODO this doesn't work. It writes out int64, not int96, and the stored schema just treats + // it as a long, not a timestamp + spark.createDataFrame(data, schema).write.parquet(dir.getCanonicalPath) + + // now we should try to read that data back. We'll fake a timezone one the table, to see + // what the resulting behavior is + ParquetRowConverter.setTimezone(TimeZone.getTimeZone("PST")) + spark.conf.set(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, false) + val readInPst = spark.read.parquet(dir.getCanonicalPath) + readInPst.explain(true) + readInPst.show() + } + } } class JobCommitFailureParquetOutputCommitter(outputPath: Path, context: TaskAttemptContext) From 69a3c8cb6c4efb35d817c214a43b217daddade4b Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 27 Jan 2017 14:18:17 -0600 Subject: [PATCH 02/40] wip --- .../datasources/parquet/ParquetIOSuite.scala | 20 ++++++++++++++----- 1 file changed, 15 insertions(+), 5 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala index b174f078141e..e7821cb3fd19 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala @@ -768,17 +768,27 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { "2016-01-01 00:39:59.123", "2016-01-01 01:29:59.123" ).map { x => Row(java.sql.Timestamp.valueOf(x)) }) - // TODO this doesn't work. It writes out int64, not int96, and the stored schema just treats - // it as a long, not a timestamp spark.createDataFrame(data, schema).write.parquet(dir.getCanonicalPath) - // now we should try to read that data back. We'll fake a timezone one the table, to see + // Ideally, we'd check the parquet schema here, make sure it was int96 + + // now we should try to read that data back. We'll fake a timezone on the table, to see // what the resulting behavior is ParquetRowConverter.setTimezone(TimeZone.getTimeZone("PST")) - spark.conf.set(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, false) +// spark.conf.set(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, false) val readInPst = spark.read.parquet(dir.getCanonicalPath) - readInPst.explain(true) readInPst.show() + + import functions._ + val originalTsCol = readInPst("timestamp") + val newTable = readInPst.withColumn("year", expr("year(timestamp)")) + assert(newTable.filter("year > '2015'").count() === 0) + + // TODO test: + // * w/ & w/out vectorization + // * filtering + // * partioning + // * DST? } } } From 51e24f28359b807f46e93975941330d5d93e3875 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 30 Jan 2017 20:55:09 -0600 Subject: [PATCH 03/40] working version for non-vectorized read -- lots of garbage too --- .../parquet/VectorizedColumnReader.java | 1 + .../parquet/ParquetReadSupport.scala | 3 +- .../parquet/ParquetRecordMaterializer.scala | 9 ++- .../parquet/ParquetRowConverter.scala | 19 +++++-- .../datasources/parquet/ParquetIOSuite.scala | 10 ++++ .../spark/sql/hive/HiveMetastoreCatalog.scala | 12 +++- .../hive/ParquetHiveCompatibilitySuite.scala | 56 ++++++++++++++++++- 7 files changed, 101 insertions(+), 9 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java index 88716e857767..d8fc73f8e39e 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java @@ -276,6 +276,7 @@ private void decodeDictionaryIds(int rowId, int num, ColumnVector column, // TODO: Convert dictionary of Binaries to dictionary of Longs if (!column.isNullAt(i)) { Binary v = dictionary.decodeToBinary(dictionaryIds.getDictId(i)); + // TODO IR XXX get tz column.putLong(i, ParquetRowConverter.binaryToSQLTimestamp(v, ParquetRowConverter.tz())); } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala index f1a35dd8a620..bf395a0bef74 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala @@ -95,7 +95,8 @@ private[parquet] class ParquetReadSupport extends ReadSupport[UnsafeRow] with Lo new ParquetRecordMaterializer( parquetRequestedSchema, ParquetReadSupport.expandUDT(catalystRequestedSchema), - new ParquetSchemaConverter(conf)) + new ParquetSchemaConverter(conf), + conf) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRecordMaterializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRecordMaterializer.scala index 4e49a0dac97c..df041996cdea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRecordMaterializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRecordMaterializer.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.datasources.parquet +import org.apache.hadoop.conf.Configuration import org.apache.parquet.io.api.{GroupConverter, RecordMaterializer} import org.apache.parquet.schema.MessageType @@ -29,13 +30,17 @@ import org.apache.spark.sql.types.StructType * @param parquetSchema Parquet schema of the records to be read * @param catalystSchema Catalyst schema of the rows to be constructed * @param schemaConverter A Parquet-Catalyst schema converter that helps initializing row converters + * @param hadoopConf hadoop Configuration for passing extra params for parquet conversion */ private[parquet] class ParquetRecordMaterializer( - parquetSchema: MessageType, catalystSchema: StructType, schemaConverter: ParquetSchemaConverter) + parquetSchema: MessageType, + catalystSchema: StructType, + schemaConverter: ParquetSchemaConverter, + hadoopConf: Configuration) extends RecordMaterializer[UnsafeRow] { private val rootConverter = - new ParquetRowConverter(schemaConverter, parquetSchema, catalystSchema, NoopUpdater) + new ParquetRowConverter(schemaConverter, parquetSchema, catalystSchema, hadoopConf, NoopUpdater) override def getCurrentRecord: UnsafeRow = rootConverter.currentRecord diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala index c2dfdeb7c600..5d08349706c5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala @@ -27,6 +27,7 @@ import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer +import org.apache.hadoop.conf.Configuration import org.apache.parquet.column.Dictionary import org.apache.parquet.io.api.{Binary, Converter, GroupConverter, PrimitiveConverter} import org.apache.parquet.schema.{GroupType, MessageType, Type} @@ -121,12 +122,14 @@ private[parquet] class ParquetPrimitiveConverter(val updater: ParentContainerUpd * @param parquetType Parquet schema of Parquet records * @param catalystType Spark SQL schema that corresponds to the Parquet record type. User-defined * types should have been expanded. + * @param hadoopConf a hadoop Configuration for passing any extra parameters for parquet conversion * @param updater An updater which propagates converted field values to the parent container */ private[parquet] class ParquetRowConverter( schemaConverter: ParquetSchemaConverter, parquetType: GroupType, catalystType: StructType, + hadoopConf: Configuration, updater: ParentContainerUpdater) extends ParquetGroupConverter(updater) with Logging { @@ -217,7 +220,7 @@ private[parquet] class ParquetRowConverter( catalystType: DataType, updater: ParentContainerUpdater): Converter with HasParentContainerUpdater = { - catalystType match { + val r = catalystType match { case BooleanType | IntegerType | LongType | FloatType | DoubleType | BinaryType => new ParquetPrimitiveConverter(updater) @@ -258,12 +261,20 @@ private[parquet] class ParquetRowConverter( case TimestampType => // TODO Implements `TIMESTAMP_MICROS` once parquet-mr has that. + val tzString = hadoopConf.get(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY) + // scalastyle:off println + println(s"creating timestamp converter w/ tzString = $tzString") + // scalastyle:on println + val tz = if (tzString == null) { + DateTimeUtils.TimeZoneGMT + } else { + TimeZone.getTimeZone(tzString) + } new ParquetPrimitiveConverter(updater) { // Converts nanosecond timestamps stored as INT96 override def addBinary(value: Binary): Unit = { - // TODO timezone from table props val timestamp = - ParquetRowConverter.binaryToSQLTimestamp(value, ParquetRowConverter.tz) + ParquetRowConverter.binaryToSQLTimestamp(value, tz) updater.setLong(timestamp) } } @@ -294,7 +305,7 @@ private[parquet] class ParquetRowConverter( case t: StructType => new ParquetRowConverter( - schemaConverter, parquetType.asGroupType(), t, new ParentContainerUpdater { + schemaConverter, parquetType.asGroupType(), t, hadoopConf, new ParentContainerUpdater { override def set(value: Any): Unit = updater.set(value.asInstanceOf[InternalRow].copy()) }) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala index e7821cb3fd19..e2364481be14 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala @@ -789,6 +789,16 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { // * filtering // * partioning // * DST? + spark.sql( + """CREATE TABLE foobar ( + | year int, + | timestamp timestamp + | ) + | STORED AS PARQUET + """.stripMargin + ) + newTable.createOrReplaceTempView("newTable") + spark.sql("insert into foobar (year, timestamp) select year, timestamp from newTable") } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index faa76b73fde4..0c832bd59838 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -205,6 +205,12 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log bucketSpec, None) val logicalRelation = cached.getOrElse { + // We add the timezone to the relation options, which automatically gets injected into + // the hadoopConf for the Parquet Converters + logInfo(s"creating HadoopFsRelation from a metastore table with" + + s" ${metastoreRelation.hiveQlTable.getParameters}") + val tzKey = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY + val tz = Option(metastoreRelation.hiveQlTable.getParameters.get(tzKey)).getOrElse("") val created = LogicalRelation( DataSource( @@ -212,7 +218,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log paths = rootPath.toString :: Nil, userSpecifiedSchema = Some(metastoreRelation.schema), bucketSpec = bucketSpec, - options = options, + options = options ++ Map(tzKey -> tz), className = fileType).resolveRelation(), catalogTable = Some(metastoreRelation.catalogTable)) @@ -237,6 +243,8 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log } private def convertToParquetRelation(relation: MetastoreRelation): LogicalRelation = { + logInfo(s"creating a parquet relation from a metastore relation on" + + s" ${relation.catalogTable.qualifiedName} : $relation") val defaultSource = new ParquetFileFormat() val fileFormatClass = classOf[ParquetFileFormat] @@ -256,10 +264,12 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log case InsertIntoTable(r: MetastoreRelation, partition, child, overwrite, ifNotExists) // Inserting into partitioned table is not supported in Parquet data source (yet). if !r.hiveQlTable.isPartitioned && shouldConvertMetastoreParquet(r) => + logInfo("checking parquet conversions for insertion") InsertIntoTable(convertToParquetRelation(r), partition, child, overwrite, ifNotExists) // Read path case relation: MetastoreRelation if shouldConvertMetastoreParquet(relation) => + logInfo(s"checking parquet conversions for $plan") val parquetRelation = convertToParquetRelation(relation) SubqueryAlias(relation.tableName, parquetRelation, None) } 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 05b6059472f5..30b1e76c0de9 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 @@ -20,9 +20,10 @@ package org.apache.spark.sql.hive import java.sql.Timestamp import org.apache.spark.sql.Row -import org.apache.spark.sql.execution.datasources.parquet.ParquetCompatibilityTest +import org.apache.spark.sql.execution.datasources.parquet.{ParquetCompatibilityTest, ParquetFileFormat} import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{StructField, StructType, TimestampType} class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHiveSingleton { /** @@ -141,4 +142,57 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi Row(Seq(Row(1))), "ARRAY>") } + + test("SPARK-12297: Parquet Timestamps & Hive Timezones") { + // Test that we can correctly adjust parquet timestamps for Hive timezone bug. + withTempPath { dir => + // First, lets generate some parquet data we can use to test this + val schema = StructType(StructField("timestamp", TimestampType) :: Nil) + // intentionally pick a few times right around new years, so time zone will effect many fields + val data = spark.sparkContext.parallelize(Seq( + "2015-12-31 23:50:59.123", + "2015-12-31 22:49:59.123", + "2016-01-01 00:39:59.123", + "2016-01-01 01:29:59.123" + ).map { x => Row(java.sql.Timestamp.valueOf(x)) }) + spark.createDataFrame(data, schema).write.parquet(dir.getCanonicalPath) + + // Ideally, we'd check the parquet schema here, make sure it was int96 + + import org.apache.spark.sql.functions._ + val readData = spark.read.parquet(dir.getCanonicalPath) + val newTable = readData.withColumn("year", expr("year(timestamp)")) + + // TODO test: + // * w/ & w/out vectorization + // * filtering + // * partioning + // * predicate pushdown + // * DST? + val key = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY + spark.sql( + raw"""CREATE TABLE foobar ( + | year int, + | timestamp timestamp + | ) + | STORED AS PARQUET + | TBLPROPERTIES ($key="America/Los_Angeles") + """.stripMargin + ) + newTable.createOrReplaceTempView("newTable") + spark.sql("INSERT INTO foobar SELECT year, timestamp FROM newTable") + + spark.conf.set(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, false) + val readFromHiveTable = spark.sql("select year, timestamp from foobar") + // from manual logging, confirmed that table properties are read back + readFromHiveTable.explain(true) + // Note that we've already stored the table with bad "year" date in this example so far + // Here we determine the year based on the table property + val collected = readFromHiveTable.withColumn("fixed_year", expr("year(timestamp)")).collect() + // Make sure our test is setup correctly + assert(collected.exists { row => row.getInt(0) == 2016 } ) + // now check we've converted the data correctly + collected.foreach { row => assert(row.getInt(2) == 2015) } + } + } } From 7e618411c83002ca098526b0691f8b184295a216 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 31 Jan 2017 13:03:34 -0600 Subject: [PATCH 04/40] working for vectorized reads -- not sure about all code paths --- .../parquet/VectorizedColumnReader.java | 20 +++++++++----- .../VectorizedParquetRecordReader.java | 10 ++++--- .../parquet/ParquetFileFormat.scala | 2 +- .../parquet/ParquetRowConverter.scala | 5 +--- .../hive/ParquetHiveCompatibilitySuite.scala | 26 +++++++++++-------- 5 files changed, 38 insertions(+), 25 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java index d8fc73f8e39e..58015932dc56 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java @@ -18,7 +18,9 @@ package org.apache.spark.sql.execution.datasources.parquet; import java.io.IOException; +import java.util.TimeZone; +import org.apache.hadoop.conf.Configuration; import org.apache.parquet.bytes.BytesUtils; import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.column.Dictionary; @@ -28,6 +30,7 @@ import org.apache.parquet.io.api.Binary; import org.apache.parquet.schema.PrimitiveType; +import org.apache.spark.sql.catalyst.util.DateTimeUtils; import org.apache.spark.sql.execution.vectorized.ColumnVector; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.DecimalType; @@ -89,11 +92,19 @@ public class VectorizedColumnReader { private final PageReader pageReader; private final ColumnDescriptor descriptor; + private final TimeZone tz; - public VectorizedColumnReader(ColumnDescriptor descriptor, PageReader pageReader) + public VectorizedColumnReader(ColumnDescriptor descriptor, PageReader pageReader, + Configuration conf) throws IOException { this.descriptor = descriptor; this.pageReader = pageReader; + String tzString = conf.get(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY()); + if (tzString == null) { + tz = DateTimeUtils.TimeZoneGMT(); + } else { + tz = TimeZone.getTimeZone(tzString); + } this.maxDefLevel = descriptor.getMaxDefinitionLevel(); DictionaryPage dictionaryPage = pageReader.readDictionaryPage(); @@ -276,9 +287,7 @@ private void decodeDictionaryIds(int rowId, int num, ColumnVector column, // TODO: Convert dictionary of Binaries to dictionary of Longs if (!column.isNullAt(i)) { Binary v = dictionary.decodeToBinary(dictionaryIds.getDictId(i)); - // TODO IR XXX get tz - column.putLong(i, - ParquetRowConverter.binaryToSQLTimestamp(v, ParquetRowConverter.tz())); + column.putLong(i, ParquetRowConverter.binaryToSQLTimestamp(v, tz)); } } } else { @@ -403,8 +412,7 @@ private void readBinaryBatch(int rowId, int num, ColumnVector column) throws IOE if (defColumn.readInteger() == maxDefLevel) { column.putLong(rowId + i, // Read 12 bytes for INT96 - ParquetRowConverter.binaryToSQLTimestamp(data.readBinary(12), - ParquetRowConverter.tz())); + ParquetRowConverter.binaryToSQLTimestamp(data.readBinary(12), tz)); } else { column.putNull(rowId + i); } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedParquetRecordReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedParquetRecordReader.java index 51bdf0f0f229..1a665990f7fa 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedParquetRecordReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedParquetRecordReader.java @@ -21,6 +21,7 @@ import java.util.Arrays; import java.util.List; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.parquet.column.ColumnDescriptor; @@ -95,6 +96,8 @@ public class VectorizedParquetRecordReader extends SpecificParquetRecordReaderBa */ private boolean returnColumnarBatch; + private Configuration conf; + /** * The default config on whether columnarBatch should be offheap. */ @@ -103,10 +106,11 @@ public class VectorizedParquetRecordReader extends SpecificParquetRecordReaderBa /** * Implementation of RecordReader API. */ - @Override - public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext) + public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext, + Configuration conf) throws IOException, InterruptedException, UnsupportedOperationException { super.initialize(inputSplit, taskAttemptContext); + this.conf = conf; initializeInternal(); } @@ -277,7 +281,7 @@ private void checkEndOfRowGroup() throws IOException { for (int i = 0; i < columns.size(); ++i) { if (missingColumns[i]) continue; columnReaders[i] = new VectorizedColumnReader(columns.get(i), - pages.getPageReader(columns.get(i))); + pages.getPageReader(columns.get(i)), conf); } totalCountLoadedSoFar += pages.getRowCount(); } 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 8a5aeff51bcf..f6b7495641ab 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 @@ -373,7 +373,7 @@ class ParquetFileFormat } val parquetReader = if (enableVectorizedReader) { val vectorizedReader = new VectorizedParquetRecordReader() - vectorizedReader.initialize(split, hadoopAttemptContext) + vectorizedReader.initialize(split, hadoopAttemptContext, broadcastedHadoopConf.value.value) logDebug(s"Appending $partitionSchema ${file.partitionValues}") vectorizedReader.initBatch(partitionSchema, file.partitionValues) if (returningBatch) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala index 5d08349706c5..448ca59953f0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala @@ -220,7 +220,7 @@ private[parquet] class ParquetRowConverter( catalystType: DataType, updater: ParentContainerUpdater): Converter with HasParentContainerUpdater = { - val r = catalystType match { + catalystType match { case BooleanType | IntegerType | LongType | FloatType | DoubleType | BinaryType => new ParquetPrimitiveConverter(updater) @@ -262,9 +262,6 @@ private[parquet] class ParquetRowConverter( case TimestampType => // TODO Implements `TIMESTAMP_MICROS` once parquet-mr has that. val tzString = hadoopConf.get(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY) - // scalastyle:off println - println(s"creating timestamp converter w/ tzString = $tzString") - // scalastyle:on println val tz = if (tzString == null) { DateTimeUtils.TimeZoneGMT } else { 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 30b1e76c0de9..65b839416cb2 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 @@ -182,17 +182,21 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi newTable.createOrReplaceTempView("newTable") spark.sql("INSERT INTO foobar SELECT year, timestamp FROM newTable") - spark.conf.set(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, false) - val readFromHiveTable = spark.sql("select year, timestamp from foobar") - // from manual logging, confirmed that table properties are read back - readFromHiveTable.explain(true) - // Note that we've already stored the table with bad "year" date in this example so far - // Here we determine the year based on the table property - val collected = readFromHiveTable.withColumn("fixed_year", expr("year(timestamp)")).collect() - // Make sure our test is setup correctly - assert(collected.exists { row => row.getInt(0) == 2016 } ) - // now check we've converted the data correctly - collected.foreach { row => assert(row.getInt(2) == 2015) } + Seq(false, true).foreach { vectorized => + withSQLConf((SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, vectorized.toString)) { + withClue(s"vectorized = $vectorized") { + val readFromHiveTable = spark.sql("select year, timestamp from foobar") + // Note that we've already stored the table with bad "year" date in this example so far + // Here we determine the year based on the table property + val collected = readFromHiveTable.withColumn("fixed_year", expr("year(timestamp)")) + .collect() + // Make sure our test is setup correctly + assert(collected.exists { row => row.getInt(0) == 2016 }) + // now check we've converted the data correctly + collected.foreach { row => assert(row.getInt(2) == 2015) } + } + } + } } } } From 9fbde13cbc431ff955564a0695c7a1c3e64e158f Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 1 Feb 2017 09:39:39 -0600 Subject: [PATCH 05/40] more tests for write path --- .../util/ParquetTimestampTimezone.scala | 21 +++++++ .../parquet/ParquetFileFormat.scala | 3 +- .../apache/spark/sql/internal/SQLConf.scala | 6 ++ .../hive/ParquetHiveCompatibilitySuite.scala | 61 ++++++++++++++++++- 4 files changed, 89 insertions(+), 2 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ParquetTimestampTimezone.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ParquetTimestampTimezone.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ParquetTimestampTimezone.scala new file mode 100644 index 000000000000..a4f7deaceb3a --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ParquetTimestampTimezone.scala @@ -0,0 +1,21 @@ +/* + * 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.catalyst.util + +object ParquetTimestampTimezone { + val PARQUET_TIMEZONE_TABLE_PROPERTY = "parquet.mr.int96.write.zone" +} 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 f6b7495641ab..1a82ef55601b 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 @@ -46,6 +46,7 @@ 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.ParquetTimestampTimezone import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ @@ -708,5 +709,5 @@ object ParquetFileFormat extends Logging { }.toOption } - val PARQUET_TIMEZONE_TABLE_PROPERTY = "parquet.mr.int96.write.zone" + val PARQUET_TIMEZONE_TABLE_PROPERTY = ParquetTimestampTimezone.PARQUET_TIMEZONE_TABLE_PROPERTY } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 5ba4192512a5..a4a071e713c8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -666,6 +666,12 @@ object SQLConf { .stringConf .createWithDefault(TimeZone.getDefault().getID()) + val PARQUET_TABLE_INCLUDE_TIMEZONE = + SQLConfigBuilder("spark.sql.session.parquet.timeZone") + .doc("""Enables inclusion of parquet timezone property in newly created parquet tables""") + .booleanConf + .createWithDefault(false) + object Deprecated { val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks" } 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 65b839416cb2..7098b446292c 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 @@ -20,6 +20,7 @@ package org.apache.spark.sql.hive import java.sql.Timestamp import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.execution.datasources.parquet.{ParquetCompatibilityTest, ParquetFileFormat} import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf @@ -143,7 +144,7 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi "ARRAY>") } - test("SPARK-12297: Parquet Timestamps & Hive Timezones") { + test("SPARK-12297: Parquet Timestamps & Hive Timezones: read path") { // Test that we can correctly adjust parquet timestamps for Hive timezone bug. withTempPath { dir => // First, lets generate some parquet data we can use to test this @@ -179,6 +180,8 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi | TBLPROPERTIES ($key="America/Los_Angeles") """.stripMargin ) + val tableMetadata = spark.sessionState.catalog.getTableMetadata(TableIdentifier("foobar")) + assert(tableMetadata.properties.get(key) === Some("America/Los_Angeles")) newTable.createOrReplaceTempView("newTable") spark.sql("INSERT INTO foobar SELECT year, timestamp FROM newTable") @@ -199,4 +202,60 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi } } } + + test("SPARK-12297: Parquet Timestamp & Hive timezones: write path") { + val key = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY + def checkHasTz(table: String, tz: Option[String]): Unit = { + val tableMetadata = spark.sessionState.catalog.getTableMetadata(TableIdentifier(table)) + assert(tableMetadata.properties.get(key) === tz) + } + Seq(false, true).foreach { shouldSetGlobalDefault => + withSQLConf((SQLConf.PARQUET_TABLE_INCLUDE_TIMEZONE.key, shouldSetGlobalDefault.toString)) { + println(s"testing w/ include tz = $shouldSetGlobalDefault") + def checkCreateTableDefaultTz(baseTable: String, explicitTz: Option[String]): Unit = { + println(s"about to enter withTable for $baseTable") + withTable(baseTable, s"like_$baseTable", s"select_$baseTable") { + println(s"inside withTable for $baseTable") + val tblProperties = explicitTz.map { tz => s"TBLPROPERTIES ($key=$tz)"}.getOrElse("") + val defaultTz = if (shouldSetGlobalDefault) Some("UTC") else None + spark.sql( + raw"""CREATE TABLE $baseTable ( + | x int + | ) + | STORED AS PARQUET + | $tblProperties + """.stripMargin + ) + val expectedTableTz = explicitTz.orElse { defaultTz } + println(s"checking table tz for base table, expecting $expectedTableTz") + checkHasTz(baseTable, expectedTableTz) + spark.sql( s"""CREATE TABLE like_$baseTable LIKE $baseTable""") + checkHasTz(s"like_$baseTable", expectedTableTz) + + spark.sql( + raw"""CREATE TABLE select_$baseTable + | STORED AS PARQUET + | AS + | SELECT * from $baseTable + """.stripMargin) + checkHasTz(s"select_$baseTable", defaultTz) + } + } + + // check creating tables a few different ways, make sure the tz property is set correctly + checkCreateTableDefaultTz("no_tz", None) + checkCreateTableDefaultTz("UTC", Some("UTC")) + checkCreateTableDefaultTz("LA", Some("America/Los Angeles")) + } + } + + // TODO create table w/ bad TZ + + // TODO insert data + + + + + pending + } } From bac9eb0ed3a65fcdaab458ef3bd52aef5af01b68 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 1 Feb 2017 14:23:07 -0600 Subject: [PATCH 06/40] expand tests; fix some metastore interaction; cleanup a lot of garbage --- .../spark/sql/execution/command/tables.scala | 6 +- .../spark/sql/hive/HiveExternalCatalog.scala | 37 ++++-- .../spark/sql/hive/MetastoreRelation.scala | 2 + .../hive/ParquetHiveCompatibilitySuite.scala | 105 +++++++++--------- 4 files changed, 86 insertions(+), 64 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 bb903a266236..f933ec058aa3 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 @@ -26,7 +26,6 @@ import scala.collection.mutable.ArrayBuffer import scala.util.control.NonFatal import scala.util.Try -import org.apache.commons.lang3.StringEscapeUtils import org.apache.hadoop.fs.Path import org.apache.spark.sql.{AnalysisException, Row, SparkSession} @@ -38,6 +37,7 @@ 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.execution.datasources.PartitioningUtils +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -70,6 +70,9 @@ case class CreateTableLikeCommand( sourceTableDesc.provider } + val properties = sourceTableDesc.properties.filter { case (k, _) => + k == ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY + } val newTableDesc = CatalogTable( identifier = targetTable, @@ -78,6 +81,7 @@ case class CreateTableLikeCommand( storage = sourceTableDesc.storage.copy(locationUri = None), schema = sourceTableDesc.schema, provider = newProvider, + properties = properties, partitionColumnNames = sourceTableDesc.partitionColumnNames, bucketSpec = sourceTableDesc.bucketSpec) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index 208c8c9d5d0c..08bfe10b6d02 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -41,8 +41,9 @@ import org.apache.spark.sql.catalyst.plans.logical.ColumnStat import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.PartitioningUtils +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.hive.client.HiveClient -import org.apache.spark.sql.internal.HiveSerDe +import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} import org.apache.spark.sql.internal.StaticSQLConf._ import org.apache.spark.sql.types.{DataType, StructType} @@ -199,37 +200,49 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat throw new TableAlreadyExistsException(db = db, table = table) } - if (tableDefinition.tableType == VIEW) { - client.createTable(tableDefinition, ignoreIfExists) + // add the extra tz property only if configured to do so, and the table does not include it + // explicitly + val tableHasTz = + tableDefinition.properties.contains(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY) + val extraTzProp = + if (conf.get(SQLConf.PARQUET_TABLE_INCLUDE_TIMEZONE) && !tableHasTz) { + Map(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY -> "UTC") + } else { + Map() + } + val tableDef = tableDefinition.copy(properties = tableDefinition.properties ++ extraTzProp) + if (tableDef.tableType == VIEW) { + client.createTable(tableDef, ignoreIfExists) } else { // Ideally we should not create a managed table with location, but Hive serde table can // specify location for managed table. And in [[CreateDataSourceTableAsSelectCommand]] we have // to create the table directory and write out data before we create this table, to avoid // exposing a partial written table. - val needDefaultTableLocation = tableDefinition.tableType == MANAGED && - tableDefinition.storage.locationUri.isEmpty + val needDefaultTableLocation = tableDef.tableType == MANAGED && + tableDef.storage.locationUri.isEmpty val tableLocation = if (needDefaultTableLocation) { - Some(defaultTablePath(tableDefinition.identifier)) + Some(defaultTablePath(tableDef.identifier)) } else { - tableDefinition.storage.locationUri + tableDef.storage.locationUri } - if (DDLUtils.isHiveTable(tableDefinition)) { - val tableWithDataSourceProps = tableDefinition.copy( + if (DDLUtils.isHiveTable(tableDef)) { + + val tableWithDataSourceProps = tableDef.copy( // We can't leave `locationUri` empty and count on Hive metastore to set a default table // location, because Hive metastore uses hive.metastore.warehouse.dir to generate default // table location for tables in default database, while we expect to use the location of // default database. - storage = tableDefinition.storage.copy(locationUri = tableLocation), + storage = tableDef.storage.copy(locationUri = tableLocation), // Here we follow data source tables and put table metadata like table schema, partition // columns etc. in table properties, so that we can work around the Hive metastore issue // about not case preserving and make Hive serde table support mixed-case column names. - properties = tableDefinition.properties ++ tableMetaToTableProps(tableDefinition)) + properties = tableDef.properties ++ tableMetaToTableProps(tableDef)) client.createTable(tableWithDataSourceProps, ignoreIfExists) } else { createDataSourceTable( - tableDefinition.withNewStorage(locationUri = tableLocation), + tableDef.withNewStorage(locationUri = tableLocation), ignoreIfExists) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala index 346757c2047a..ab824d4122f9 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala @@ -36,6 +36,8 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.{AttributeMap, AttributeReference, Expression} import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics} import org.apache.spark.sql.execution.FileRelation +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructField 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 7098b446292c..f66c1db9954b 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 @@ -19,11 +19,14 @@ package org.apache.spark.sql.hive import java.sql.Timestamp -import org.apache.spark.sql.Row +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.execution.datasources.parquet.{ParquetCompatibilityTest, ParquetFileFormat} -import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.hive.test.{TestHiveContext, TestHiveSingleton} import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.StaticSQLConf._ +import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types.{StructField, StructType, TimestampType} class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHiveSingleton { @@ -139,12 +142,13 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi } test("SPARK-16344: array of struct with a single field named 'array_element'") { + testParquetHiveCompatibility( Row(Seq(Row(1))), "ARRAY>") } - test("SPARK-12297: Parquet Timestamps & Hive Timezones: read path") { + ignore("SPARK-12297: Parquet Timestamps & Hive Timezones: read path") { // Test that we can correctly adjust parquet timestamps for Hive timezone bug. withTempPath { dir => // First, lets generate some parquet data we can use to test this @@ -203,59 +207,58 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi } } - test("SPARK-12297: Parquet Timestamp & Hive timezones: write path") { - val key = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY - def checkHasTz(table: String, tz: Option[String]): Unit = { - val tableMetadata = spark.sessionState.catalog.getTableMetadata(TableIdentifier(table)) - assert(tableMetadata.properties.get(key) === tz) - } - Seq(false, true).foreach { shouldSetGlobalDefault => - withSQLConf((SQLConf.PARQUET_TABLE_INCLUDE_TIMEZONE.key, shouldSetGlobalDefault.toString)) { - println(s"testing w/ include tz = $shouldSetGlobalDefault") - def checkCreateTableDefaultTz(baseTable: String, explicitTz: Option[String]): Unit = { - println(s"about to enter withTable for $baseTable") - withTable(baseTable, s"like_$baseTable", s"select_$baseTable") { - println(s"inside withTable for $baseTable") - val tblProperties = explicitTz.map { tz => s"TBLPROPERTIES ($key=$tz)"}.getOrElse("") - val defaultTz = if (shouldSetGlobalDefault) Some("UTC") else None - spark.sql( - raw"""CREATE TABLE $baseTable ( - | x int - | ) - | STORED AS PARQUET - | $tblProperties + Seq(false, true).foreach { setTableTzByDefault => + // we're cheating here, and changing the conf at runtime + test(s"SPARK-12297: Parquet Timestamp & Hive timezones write path, " + + s"default = $setTableTzByDefault") { + sparkContext.conf.set( + SQLConf.PARQUET_TABLE_INCLUDE_TIMEZONE.key, setTableTzByDefault.toString) + val key = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY + def checkHasTz(table: String, tz: Option[String]): Unit = { + val tableMetadata = spark.sessionState.catalog.getTableMetadata(TableIdentifier(table)) + assert(tableMetadata.properties.get(key) === tz) + } + println(s"$setTableTzByDefault;" + + s" ${sparkContext.conf.get(SQLConf.PARQUET_TABLE_INCLUDE_TIMEZONE)}") + def checkCreateTableDefaultTz(baseTable: String, explicitTz: Option[String]): Unit = { + withTable(baseTable, s"like_$baseTable", s"select_$baseTable") { + val tblProperties = explicitTz.map { + tz => raw"""TBLPROPERTIES ($key="$tz")""" + }.getOrElse("") + val defaultTz = if (setTableTzByDefault) Some("UTC") else None + val stmt = + raw"""CREATE TABLE $baseTable ( + | x int + | ) + | STORED AS PARQUET + | $tblProperties """.stripMargin - ) - val expectedTableTz = explicitTz.orElse { defaultTz } - println(s"checking table tz for base table, expecting $expectedTableTz") - checkHasTz(baseTable, expectedTableTz) - spark.sql( s"""CREATE TABLE like_$baseTable LIKE $baseTable""") - checkHasTz(s"like_$baseTable", expectedTableTz) - - spark.sql( - raw"""CREATE TABLE select_$baseTable - | STORED AS PARQUET - | AS - | SELECT * from $baseTable - """.stripMargin) + println(stmt) + spark.sql(stmt) + val expectedTableTz = explicitTz.orElse(defaultTz) + println(s"checking table tz for $baseTable, expecting $expectedTableTz") + checkHasTz(baseTable, expectedTableTz) + spark.sql(s"""CREATE TABLE like_$baseTable LIKE $baseTable""") + checkHasTz(s"like_$baseTable", expectedTableTz) + spark.sql( + raw"""CREATE TABLE select_$baseTable + | STORED AS PARQUET + | AS + | SELECT * from $baseTable + """.stripMargin) checkHasTz(s"select_$baseTable", defaultTz) - } } - - // check creating tables a few different ways, make sure the tz property is set correctly - checkCreateTableDefaultTz("no_tz", None) - checkCreateTableDefaultTz("UTC", Some("UTC")) - checkCreateTableDefaultTz("LA", Some("America/Los Angeles")) } - } + // check creating tables a few different ways, make sure the tz property is set correctly + checkCreateTableDefaultTz("no_tz", None) - // TODO create table w/ bad TZ + checkCreateTableDefaultTz("UTC", Some("UTC")) + checkCreateTableDefaultTz("LA", Some("America/Los Angeles")) + // TODO create table w/ bad TZ - // TODO insert data - - - - - pending + // TODO insert data + pending + } } + } From 1b05978dc9ee1de5f6d1d7031510ab6b91a6e5b9 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 1 Feb 2017 14:49:31 -0600 Subject: [PATCH 07/40] more cleanup --- .../apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala | 2 -- 1 file changed, 2 deletions(-) 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 f66c1db9954b..9eb49fdb8514 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 @@ -218,8 +218,6 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi val tableMetadata = spark.sessionState.catalog.getTableMetadata(TableIdentifier(table)) assert(tableMetadata.properties.get(key) === tz) } - println(s"$setTableTzByDefault;" + - s" ${sparkContext.conf.get(SQLConf.PARQUET_TABLE_INCLUDE_TIMEZONE)}") def checkCreateTableDefaultTz(baseTable: String, explicitTz: Option[String]): Unit = { withTable(baseTable, s"like_$baseTable", s"select_$baseTable") { val tblProperties = explicitTz.map { From b622d278d7a451846dcde28ed01c9618b7a00662 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 1 Feb 2017 16:03:38 -0600 Subject: [PATCH 08/40] handle bad timezones; include unit test --- .../sql/catalyst/util/DateTimeUtils.scala | 5 +++ .../spark/sql/hive/HiveExternalCatalog.scala | 13 ++++-- .../hive/ParquetHiveCompatibilitySuite.scala | 44 ++++++++++++------- 3 files changed, 41 insertions(+), 21 deletions(-) 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 af70efbb0a91..74efa44a6b16 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 @@ -473,6 +473,11 @@ object DateTimeUtils { false } + lazy val validTimezones = TimeZone.getAvailableIDs().toSet + def isValidTimezone(timezoneId: String): Boolean = { + validTimezones.contains(timezoneId) + } + /** * Returns the microseconds since year zero (-17999) from microseconds since epoch. */ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index 08bfe10b6d02..f4a341530e59 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -38,7 +38,7 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.escapePathName import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.ColumnStat -import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.PartitioningUtils import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat @@ -202,10 +202,15 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat // add the extra tz property only if configured to do so, and the table does not include it // explicitly - val tableHasTz = - tableDefinition.properties.contains(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY) + val tableTz = tableDefinition.properties.get(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY) + tableTz.foreach { tz => + if (!DateTimeUtils.isValidTimezone(tz)) { + throw new AnalysisException(s"Cannot set" + + s" ${ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY} to invalid timezone $tz") + } + } val extraTzProp = - if (conf.get(SQLConf.PARQUET_TABLE_INCLUDE_TIMEZONE) && !tableHasTz) { + if (conf.get(SQLConf.PARQUET_TABLE_INCLUDE_TIMEZONE) && tableTz.isEmpty) { Map(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY -> "UTC") } else { Map() 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 9eb49fdb8514..8fb59e23df1c 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 @@ -19,8 +19,8 @@ package org.apache.spark.sql.hive import java.sql.Timestamp -import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} -import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark._ +import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.execution.datasources.parquet.{ParquetCompatibilityTest, ParquetFileFormat} import org.apache.spark.sql.hive.test.{TestHiveContext, TestHiveSingleton} @@ -207,10 +207,11 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi } } - Seq(false, true).foreach { setTableTzByDefault => - // we're cheating here, and changing the conf at runtime - test(s"SPARK-12297: Parquet Timestamp & Hive timezones write path, " + - s"default = $setTableTzByDefault") { + test(s"SPARK-12297: Parquet Timestamp & Hive timezones write path") { + Seq(false, true).foreach { setTableTzByDefault => + // we're cheating a bit here, in general SparkConf isn't meant to be set at runtime, + // but its OK in this case, and lets us run this test, because these tests don't like + // creating multiple HiveContexts in the same jvm sparkContext.conf.set( SQLConf.PARQUET_TABLE_INCLUDE_TIMEZONE.key, setTableTzByDefault.toString) val key = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY @@ -224,17 +225,13 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi tz => raw"""TBLPROPERTIES ($key="$tz")""" }.getOrElse("") val defaultTz = if (setTableTzByDefault) Some("UTC") else None - val stmt = - raw"""CREATE TABLE $baseTable ( - | x int - | ) - | STORED AS PARQUET - | $tblProperties - """.stripMargin - println(stmt) - spark.sql(stmt) + spark.sql(raw"""CREATE TABLE $baseTable ( + | x int + | ) + | STORED AS PARQUET + | $tblProperties + """.stripMargin) val expectedTableTz = explicitTz.orElse(defaultTz) - println(s"checking table tz for $baseTable, expecting $expectedTableTz") checkHasTz(baseTable, expectedTableTz) spark.sql(s"""CREATE TABLE like_$baseTable LIKE $baseTable""") checkHasTz(s"like_$baseTable", expectedTableTz) @@ -251,9 +248,22 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi checkCreateTableDefaultTz("no_tz", None) checkCreateTableDefaultTz("UTC", Some("UTC")) - checkCreateTableDefaultTz("LA", Some("America/Los Angeles")) + checkCreateTableDefaultTz("LA", Some("America/Los_Angeles")) // TODO create table w/ bad TZ + val badTzException = intercept[AnalysisException] { + spark.sql( + raw"""CREATE TABLE bad_tz_table ( + | x int + | ) + | STORED AS PARQUET + | TBLPROPERTIES ($key="Blart Versenwald III") + """.stripMargin) + } + assert(badTzException.getMessage.contains("Blart Versenwald III")) + + // TODO check on an ALTER TABLE + // TODO insert data pending } From 0604403e0d67d59c9c586b72b340db5c157d817b Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 1 Feb 2017 23:43:08 -0600 Subject: [PATCH 09/40] write support; lots more unit tests --- .../parquet/ParquetWriteSupport.scala | 22 ++- .../hive/ParquetHiveCompatibilitySuite.scala | 129 +++++++++++++++--- 2 files changed, 130 insertions(+), 21 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala index a31d2b9c37e9..b8e7792ad302 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.datasources.parquet import java.nio.{ByteBuffer, ByteOrder} import java.util +import java.util.TimeZone import scala.collection.JavaConverters.mapAsJavaMapConverter @@ -72,6 +73,8 @@ private[parquet] class ParquetWriteSupport extends WriteSupport[InternalRow] wit // Reusable byte array used to write decimal values private val decimalBuffer = new Array[Byte](minBytesForPrecision(DecimalType.MAX_PRECISION)) + private var timezone: TimeZone = _ + override def init(configuration: Configuration): WriteContext = { val schemaString = configuration.get(ParquetWriteSupport.SPARK_ROW_SCHEMA) this.schema = StructType.fromString(schemaString) @@ -81,6 +84,12 @@ private[parquet] class ParquetWriteSupport extends WriteSupport[InternalRow] wit configuration.get(SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key).toBoolean } this.rootFieldWriters = schema.map(_.dataType).map(makeWriter) + val tzString = configuration.get(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY) + timezone = if (tzString == null) { + DateTimeUtils.TimeZoneGMT + } else { + TimeZone.getTimeZone(tzString) + } val messageType = new ParquetSchemaConverter(configuration).convert(schema) val metadata = Map(ParquetReadSupport.SPARK_METADATA_KEY -> schemaString).asJava @@ -163,7 +172,18 @@ private[parquet] class ParquetWriteSupport extends WriteSupport[InternalRow] wit // NOTE: Starting from Spark 1.5, Spark SQL `TimestampType` only has microsecond // precision. Nanosecond parts of timestamp values read from INT96 are simply stripped. - val (julianDay, timeOfDayNanos) = DateTimeUtils.toJulianDay(row.getLong(ordinal)) + val rawMicros = row.getLong(ordinal) + val adjustedMicros = if (timezone == DateTimeUtils.TimeZoneGMT) { + rawMicros + } else { + // For compatibility with HIVE-12767, always write data in UTC, so adjust *from* the + // table timezone *to* UTC. Eg., if the table was created with TZ = America/Los_Angeles + // (UTC-8 w/out DST), then add 8 hours to the timestamp when we save. + val millisOffset = timezone.getOffset(rawMicros / 1000) + // scalstyle:off + rawMicros - (millisOffset * 1000L) + } + val (julianDay, timeOfDayNanos) = DateTimeUtils.toJulianDay(adjustedMicros) val buf = ByteBuffer.wrap(timestampBuffer) buf.order(ByteOrder.LITTLE_ENDIAN).putLong(timeOfDayNanos).putInt(julianDay) recordConsumer.addBinary(Binary.fromReusedByteArray(timestampBuffer)) 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 8fb59e23df1c..1408c8d204a1 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,7 +17,9 @@ package org.apache.spark.sql.hive +import java.io.File import java.sql.Timestamp +import java.util.TimeZone import org.apache.spark._ import org.apache.spark.sql.{AnalysisException, Row, SparkSession} @@ -27,7 +29,7 @@ import org.apache.spark.sql.hive.test.{TestHiveContext, TestHiveSingleton} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf._ import org.apache.spark.sql.test.SQLTestUtils -import org.apache.spark.sql.types.{StructField, StructType, TimestampType} +import org.apache.spark.sql.types.{StringType, StructField, StructType, TimestampType} class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHiveSingleton { /** @@ -207,7 +209,8 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi } } - test(s"SPARK-12297: Parquet Timestamp & Hive timezones write path") { + // TODO GMT / UTC should both use fast path + test(s"SPARK-12297: Parquet Timestamp & Hive timezone") { Seq(false, true).foreach { setTableTzByDefault => // we're cheating a bit here, in general SparkConf isn't meant to be set at runtime, // but its OK in this case, and lets us run this test, because these tests don't like @@ -219,37 +222,127 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi val tableMetadata = spark.sessionState.catalog.getTableMetadata(TableIdentifier(table)) assert(tableMetadata.properties.get(key) === tz) } - def checkCreateTableDefaultTz(baseTable: String, explicitTz: Option[String]): Unit = { - withTable(baseTable, s"like_$baseTable", s"select_$baseTable") { + def checkCreateReadWrite(baseTable: String, explicitTz: Option[String]): Unit = { + withTable(baseTable, s"like_$baseTable", s"select_$baseTable", s"external_$baseTable", + s"saveAsTable_$baseTable", s"insert_$baseTable") { + + // check that created tables have correct TBLPROPERTIES val tblProperties = explicitTz.map { tz => raw"""TBLPROPERTIES ($key="$tz")""" }.getOrElse("") val defaultTz = if (setTableTzByDefault) Some("UTC") else None - spark.sql(raw"""CREATE TABLE $baseTable ( - | x int - | ) - | STORED AS PARQUET - | $tblProperties + spark.sql( + raw"""CREATE TABLE $baseTable ( + | x int + | ) + | STORED AS PARQUET + | $tblProperties """.stripMargin) val expectedTableTz = explicitTz.orElse(defaultTz) checkHasTz(baseTable, expectedTableTz) spark.sql(s"""CREATE TABLE like_$baseTable LIKE $baseTable""") checkHasTz(s"like_$baseTable", expectedTableTz) spark.sql( - raw"""CREATE TABLE select_$baseTable + raw"""CREATE TABLE select_$baseTable | STORED AS PARQUET | AS | SELECT * from $baseTable - """.stripMargin) - checkHasTz(s"select_$baseTable", defaultTz) + """.stripMargin) + checkHasTz(s"select_$baseTable", defaultTz) + + // TODO check on an ALTER TABLE + + import spark.implicits._ + val rawData = spark.createDataset(Seq( + "2015-12-31 23:50:59.123", + "2015-12-31 22:49:59.123", + "2016-01-01 00:39:59.123", + "2016-01-01 01:29:59.123" + ).map { x => java.sql.Timestamp.valueOf(x) }) + + // Check writing data out. + // We write data into our tables, and then check the raw parquet files to see whether + // the correct conversion was applied. + rawData.write.saveAsTable(s"saveAsTable_$baseTable") + checkHasTz(s"saveAsTable_$baseTable", defaultTz) + spark.sql( + raw"""CREATE TABLE insert_$baseTable ( + | ts timestamp + | ) + | STORED AS PARQUET + | $tblProperties + """.stripMargin) + checkHasTz(s"insert_$baseTable", expectedTableTz) + rawData.createOrReplaceTempView(s"tempView_$baseTable") + spark.sql(s"INSERT INTO insert_$baseTable SELECT value AS ts FROM tempView_$baseTable") + val readFromTable = spark.table(s"insert_$baseTable").collect().map(_.getAs[Timestamp](0)) + // no matter what, roundtripping via the table should leave the data unchanged + assert(readFromTable === rawData.collect()) + // lets load the raw parquet data on disk, and check if it was adjusted correctly + val onDiskLocation = """file:(.*)""".r.findFirstMatchIn(spark.sessionState.catalog + .getTableMetadata(TableIdentifier(s"insert_$baseTable")).location).get.group(1) + val readFromDisk = spark.read.parquet(onDiskLocation).collect().map(_.getAs[Timestamp](0)) + val expectedReadFromDisk = expectedTableTz match { + case Some("America/Los_Angeles") => + // we should take a timestamp that is assumed to be in LA time, and shift it to UTC + // by subtracting the offset from UTC. + val tz = TimeZone.getTimeZone("America/Los_Angeles") + rawData.collect().map { ts => + new Timestamp(ts.getTime - tz.getOffset(ts.getTime)) + } + case _ => + rawData.collect() + } + assert(readFromDisk === expectedReadFromDisk, readFromDisk.mkString(",")) + + // check reading data back in + // TODO check predicate pushdown + // we intentionally save this data directly, without creating a table, so we can + // see that the data is read back differently depending on table properties + withTempPath { path => + rawData.write.parquet(path.getCanonicalPath) + val options = Map("path" -> path.getCanonicalPath) ++ + explicitTz.map { tz => Map(key -> tz) }.getOrElse(Map()) + + spark.catalog.createTable( + tableName = s"external_$baseTable", + source = "parquet", + schema = new StructType().add("value", TimestampType), + options = options + ) + Seq(false, true).foreach { vectorized => + withSQLConf((SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, vectorized.toString)) { + withClue(s"vectorized = $vectorized") { + val collectedFromExternal = + spark.sql(s"select value from external_$baseTable").collect() + .map(_.getAs[Timestamp](0)) + val expTimestamps = explicitTz match { + case Some("America/Los_Angeles") => + // this is assuming that the times were adjusted *from* LA *to* UTC when they + // were written out, so on reading we do the reverse. Eg., originally the + // data was 12pm, but it got written out as 8pm. We need to read it back + // as 12pm. The offset is stored as the difference from UTC, so in this case + // it would be -8. + val tz = TimeZone.getTimeZone("America/Los_Angeles") + rawData.collect().map { ts => + new Timestamp(ts.getTime + tz.getOffset(ts.getTime)) + } + case _ => + // no modification to raw data in parquet + rawData.collect() + } + assert(collectedFromExternal === expTimestamps) + } + } + } + } } } // check creating tables a few different ways, make sure the tz property is set correctly - checkCreateTableDefaultTz("no_tz", None) + checkCreateReadWrite("no_tz", None) - checkCreateTableDefaultTz("UTC", Some("UTC")) - checkCreateTableDefaultTz("LA", Some("America/Los_Angeles")) - // TODO create table w/ bad TZ + checkCreateReadWrite("UTC", Some("UTC")) + checkCreateReadWrite("LA", Some("America/Los_Angeles")) val badTzException = intercept[AnalysisException] { spark.sql( @@ -262,10 +355,6 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi } assert(badTzException.getMessage.contains("Blart Versenwald III")) - // TODO check on an ALTER TABLE - - // TODO insert data - pending } } From f45516da3ee5adf6300085a807b7acd4193cbb36 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 2 Feb 2017 10:17:39 -0600 Subject: [PATCH 10/40] add tests for alter table --- .../hive/ParquetHiveCompatibilitySuite.scala | 287 ++++++++---------- 1 file changed, 124 insertions(+), 163 deletions(-) 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 1408c8d204a1..f73c2fd1cf6c 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 @@ -150,67 +150,14 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi "ARRAY>") } - ignore("SPARK-12297: Parquet Timestamps & Hive Timezones: read path") { - // Test that we can correctly adjust parquet timestamps for Hive timezone bug. - withTempPath { dir => - // First, lets generate some parquet data we can use to test this - val schema = StructType(StructField("timestamp", TimestampType) :: Nil) - // intentionally pick a few times right around new years, so time zone will effect many fields - val data = spark.sparkContext.parallelize(Seq( - "2015-12-31 23:50:59.123", - "2015-12-31 22:49:59.123", - "2016-01-01 00:39:59.123", - "2016-01-01 01:29:59.123" - ).map { x => Row(java.sql.Timestamp.valueOf(x)) }) - spark.createDataFrame(data, schema).write.parquet(dir.getCanonicalPath) - - // Ideally, we'd check the parquet schema here, make sure it was int96 - - import org.apache.spark.sql.functions._ - val readData = spark.read.parquet(dir.getCanonicalPath) - val newTable = readData.withColumn("year", expr("year(timestamp)")) - - // TODO test: - // * w/ & w/out vectorization - // * filtering - // * partioning - // * predicate pushdown - // * DST? - val key = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY - spark.sql( - raw"""CREATE TABLE foobar ( - | year int, - | timestamp timestamp - | ) - | STORED AS PARQUET - | TBLPROPERTIES ($key="America/Los_Angeles") - """.stripMargin - ) - val tableMetadata = spark.sessionState.catalog.getTableMetadata(TableIdentifier("foobar")) - assert(tableMetadata.properties.get(key) === Some("America/Los_Angeles")) - newTable.createOrReplaceTempView("newTable") - spark.sql("INSERT INTO foobar SELECT year, timestamp FROM newTable") - - Seq(false, true).foreach { vectorized => - withSQLConf((SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, vectorized.toString)) { - withClue(s"vectorized = $vectorized") { - val readFromHiveTable = spark.sql("select year, timestamp from foobar") - // Note that we've already stored the table with bad "year" date in this example so far - // Here we determine the year based on the table property - val collected = readFromHiveTable.withColumn("fixed_year", expr("year(timestamp)")) - .collect() - // Make sure our test is setup correctly - assert(collected.exists { row => row.getInt(0) == 2016 }) - // now check we've converted the data correctly - collected.foreach { row => assert(row.getInt(2) == 2015) } - } - } - } - } - } - // TODO GMT / UTC should both use fast path test(s"SPARK-12297: Parquet Timestamp & Hive timezone") { + // Check creating parquet tables, writing data into them, and reading it back out under a + // variety of conditions: + // * global conf for setting table tz by default + // * tables with explicit tz and those without + // * altering table properties directly + // * UTC and non-UTC timezones Seq(false, true).foreach { setTableTzByDefault => // we're cheating a bit here, in general SparkConf isn't meant to be set at runtime, // but its OK in this case, and lets us run this test, because these tests don't like @@ -225,120 +172,134 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi def checkCreateReadWrite(baseTable: String, explicitTz: Option[String]): Unit = { withTable(baseTable, s"like_$baseTable", s"select_$baseTable", s"external_$baseTable", s"saveAsTable_$baseTable", s"insert_$baseTable") { - - // check that created tables have correct TBLPROPERTIES - val tblProperties = explicitTz.map { - tz => raw"""TBLPROPERTIES ($key="$tz")""" - }.getOrElse("") - val defaultTz = if (setTableTzByDefault) Some("UTC") else None - spark.sql( - raw"""CREATE TABLE $baseTable ( - | x int - | ) - | STORED AS PARQUET - | $tblProperties - """.stripMargin) - val expectedTableTz = explicitTz.orElse(defaultTz) - checkHasTz(baseTable, expectedTableTz) - spark.sql(s"""CREATE TABLE like_$baseTable LIKE $baseTable""") - checkHasTz(s"like_$baseTable", expectedTableTz) - spark.sql( - raw"""CREATE TABLE select_$baseTable - | STORED AS PARQUET - | AS - | SELECT * from $baseTable + withClue(s"table tz default=$setTableTzByDefault; explicitTz = $explicitTz;") { + // check that created tables have correct TBLPROPERTIES + val tblProperties = explicitTz.map { + tz => raw"""TBLPROPERTIES ($key="$tz")""" + }.getOrElse("") + val defaultTz = if (setTableTzByDefault) Some("UTC") else None + spark.sql( + raw"""CREATE TABLE $baseTable ( + | x int + | ) + | STORED AS PARQUET + | $tblProperties """.stripMargin) - checkHasTz(s"select_$baseTable", defaultTz) - - // TODO check on an ALTER TABLE - - import spark.implicits._ - val rawData = spark.createDataset(Seq( - "2015-12-31 23:50:59.123", - "2015-12-31 22:49:59.123", - "2016-01-01 00:39:59.123", - "2016-01-01 01:29:59.123" - ).map { x => java.sql.Timestamp.valueOf(x) }) - - // Check writing data out. - // We write data into our tables, and then check the raw parquet files to see whether - // the correct conversion was applied. - rawData.write.saveAsTable(s"saveAsTable_$baseTable") - checkHasTz(s"saveAsTable_$baseTable", defaultTz) - spark.sql( - raw"""CREATE TABLE insert_$baseTable ( - | ts timestamp - | ) + val expectedTableTz = explicitTz.orElse(defaultTz) + checkHasTz(baseTable, expectedTableTz) + spark.sql(s"CREATE TABLE like_$baseTable LIKE $baseTable") + checkHasTz(s"like_$baseTable", expectedTableTz) + spark.sql( + raw"""CREATE TABLE select_$baseTable | STORED AS PARQUET - | $tblProperties + | AS + | SELECT * from $baseTable + """.stripMargin) + checkHasTz(s"select_$baseTable", defaultTz) + + // check alter table, setting, unsetting, resetting the property + spark.sql( + raw"""ALTER TABLE $baseTable SET TBLPROPERTIES ($key="America/Los_Angeles")""") + checkHasTz(baseTable, Some("America/Los_Angeles")) + spark.sql(raw"""ALTER TABLE $baseTable SET TBLPROPERTIES ($key="UTC")""") + checkHasTz(baseTable, Some("UTC")) + spark.sql(raw"""ALTER TABLE $baseTable UNSET TBLPROPERTIES ($key)""") + checkHasTz(baseTable, None) + explicitTz.foreach { tz => + spark.sql( raw"""ALTER TABLE $baseTable SET TBLPROPERTIES ($key="$tz")""") + checkHasTz(baseTable, expectedTableTz) + } + + + import spark.implicits._ + val rawData = spark.createDataset(Seq( + "2015-12-31 23:50:59.123", + "2015-12-31 22:49:59.123", + "2016-01-01 00:39:59.123", + "2016-01-01 01:29:59.123" + ).map { x => java.sql.Timestamp.valueOf(x) }) + + // Check writing data out. + // We write data into our tables, and then check the raw parquet files to see whether + // the correct conversion was applied. + rawData.write.saveAsTable(s"saveAsTable_$baseTable") + checkHasTz(s"saveAsTable_$baseTable", defaultTz) + spark.sql( + raw"""CREATE TABLE insert_$baseTable ( + | ts timestamp + | ) + | STORED AS PARQUET + | $tblProperties """.stripMargin) - checkHasTz(s"insert_$baseTable", expectedTableTz) - rawData.createOrReplaceTempView(s"tempView_$baseTable") - spark.sql(s"INSERT INTO insert_$baseTable SELECT value AS ts FROM tempView_$baseTable") - val readFromTable = spark.table(s"insert_$baseTable").collect().map(_.getAs[Timestamp](0)) - // no matter what, roundtripping via the table should leave the data unchanged - assert(readFromTable === rawData.collect()) - // lets load the raw parquet data on disk, and check if it was adjusted correctly - val onDiskLocation = """file:(.*)""".r.findFirstMatchIn(spark.sessionState.catalog - .getTableMetadata(TableIdentifier(s"insert_$baseTable")).location).get.group(1) - val readFromDisk = spark.read.parquet(onDiskLocation).collect().map(_.getAs[Timestamp](0)) - val expectedReadFromDisk = expectedTableTz match { - case Some("America/Los_Angeles") => - // we should take a timestamp that is assumed to be in LA time, and shift it to UTC - // by subtracting the offset from UTC. - val tz = TimeZone.getTimeZone("America/Los_Angeles") - rawData.collect().map { ts => - new Timestamp(ts.getTime - tz.getOffset(ts.getTime)) - } - case _ => - rawData.collect() - } - assert(readFromDisk === expectedReadFromDisk, readFromDisk.mkString(",")) - - // check reading data back in - // TODO check predicate pushdown - // we intentionally save this data directly, without creating a table, so we can - // see that the data is read back differently depending on table properties - withTempPath { path => - rawData.write.parquet(path.getCanonicalPath) - val options = Map("path" -> path.getCanonicalPath) ++ - explicitTz.map { tz => Map(key -> tz) }.getOrElse(Map()) - - spark.catalog.createTable( - tableName = s"external_$baseTable", - source = "parquet", - schema = new StructType().add("value", TimestampType), - options = options - ) - Seq(false, true).foreach { vectorized => - withSQLConf((SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, vectorized.toString)) { - withClue(s"vectorized = $vectorized") { - val collectedFromExternal = - spark.sql(s"select value from external_$baseTable").collect() - .map(_.getAs[Timestamp](0)) - val expTimestamps = explicitTz match { - case Some("America/Los_Angeles") => - // this is assuming that the times were adjusted *from* LA *to* UTC when they - // were written out, so on reading we do the reverse. Eg., originally the - // data was 12pm, but it got written out as 8pm. We need to read it back - // as 12pm. The offset is stored as the difference from UTC, so in this case - // it would be -8. - val tz = TimeZone.getTimeZone("America/Los_Angeles") - rawData.collect().map { ts => - new Timestamp(ts.getTime + tz.getOffset(ts.getTime)) - } - case _ => - // no modification to raw data in parquet - rawData.collect() + checkHasTz(s"insert_$baseTable", expectedTableTz) + rawData.createOrReplaceTempView(s"tempView_$baseTable") + spark.sql(s"INSERT INTO insert_$baseTable SELECT value AS ts FROM tempView_$baseTable") + val readFromTable = spark.table(s"insert_$baseTable").collect() + .map(_.getAs[Timestamp](0)) + // no matter what, roundtripping via the table should leave the data unchanged + assert(readFromTable === rawData.collect()) + // lets load the raw parquet data on disk, and check if it was adjusted correctly + val onDiskLocation = """file:(.*)""".r.findFirstMatchIn(spark.sessionState.catalog + .getTableMetadata(TableIdentifier(s"insert_$baseTable")).location).get.group(1) + val readFromDisk = spark.read.parquet(onDiskLocation).collect() + .map(_.getAs[Timestamp](0)) + val expectedReadFromDisk = expectedTableTz match { + case Some("America/Los_Angeles") => + // we should take a timestamp that is assumed to be in LA time, and shift it to UTC + // by subtracting the offset from UTC. + val tz = TimeZone.getTimeZone("America/Los_Angeles") + rawData.collect().map { ts => + new Timestamp(ts.getTime - tz.getOffset(ts.getTime)) + } + case _ => + rawData.collect() + } + assert(readFromDisk === expectedReadFromDisk, readFromDisk.mkString(",")) + + // check reading data back in + // TODO check predicate pushdown + // we intentionally save this data directly, without creating a table, so we can + // see that the data is read back differently depending on table properties + withTempPath { path => + rawData.write.parquet(path.getCanonicalPath) + val options = Map("path" -> path.getCanonicalPath) ++ + explicitTz.map { tz => Map(key -> tz) }.getOrElse(Map()) + + spark.catalog.createTable( + tableName = s"external_$baseTable", + source = "parquet", + schema = new StructType().add("value", TimestampType), + options = options + ) + Seq(false, true).foreach { vectorized => + withSQLConf((SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, vectorized.toString)) { + withClue(s"vectorized = $vectorized;") { + val collectedFromExternal = + spark.sql(s"select value from external_$baseTable").collect() + .map(_.getAs[Timestamp](0)) + val expTimestamps = explicitTz match { + case Some("America/Los_Angeles") => + // this is assuming that the times were adjusted *from* LA *to* UTC when + // they were written out, so on reading we do the reverse. Eg., originally + // the data was 12pm, but it got written out as 8pm. We need to read it + // back as 12pm. The offset is stored as the difference from UTC, so in + // this case it would be -8. + val tz = TimeZone.getTimeZone("America/Los_Angeles") + rawData.collect().map { ts => + new Timestamp(ts.getTime + tz.getOffset(ts.getTime)) + } + case _ => + // no modification to raw data in parquet + rawData.collect() + } + assert(collectedFromExternal === expTimestamps) } - assert(collectedFromExternal === expTimestamps) } } } } } } - // check creating tables a few different ways, make sure the tz property is set correctly checkCreateReadWrite("no_tz", None) checkCreateReadWrite("UTC", Some("UTC")) From d4511a68a881c0f2b1238d644e4e6fa1f5578154 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 2 Feb 2017 10:25:52 -0600 Subject: [PATCH 11/40] utc or gmt; cleanup --- .../sql/catalyst/util/DateTimeUtils.scala | 4 ++++ .../util/ParquetTimestampTimezone.scala | 21 ------------------- .../parquet/ParquetFileFormat.scala | 3 +-- .../parquet/ParquetRowConverter.scala | 8 +------ .../parquet/ParquetWriteSupport.scala | 2 +- .../spark/sql/hive/MetastoreRelation.scala | 2 -- 6 files changed, 7 insertions(+), 33 deletions(-) delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ParquetTimestampTimezone.scala 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 74efa44a6b16..cd2b93efdfa2 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 @@ -478,6 +478,10 @@ object DateTimeUtils { validTimezones.contains(timezoneId) } + def isUtcOrGmt(timeZone: TimeZone): Boolean = { + timeZone.getID == "UTC" || timeZone.getID() == "GMT" + } + /** * Returns the microseconds since year zero (-17999) from microseconds since epoch. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ParquetTimestampTimezone.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ParquetTimestampTimezone.scala deleted file mode 100644 index a4f7deaceb3a..000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ParquetTimestampTimezone.scala +++ /dev/null @@ -1,21 +0,0 @@ -/* - * 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.catalyst.util - -object ParquetTimestampTimezone { - val PARQUET_TIMEZONE_TABLE_PROPERTY = "parquet.mr.int96.write.zone" -} 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 1a82ef55601b..f6b7495641ab 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 @@ -46,7 +46,6 @@ 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.ParquetTimestampTimezone import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ @@ -709,5 +708,5 @@ object ParquetFileFormat extends Logging { }.toOption } - val PARQUET_TIMEZONE_TABLE_PROPERTY = ParquetTimestampTimezone.PARQUET_TIMEZONE_TABLE_PROPERTY + val PARQUET_TIMEZONE_TABLE_PROPERTY = "parquet.mr.int96.write.zone" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala index 448ca59953f0..6b67a7058696 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala @@ -652,12 +652,6 @@ private[parquet] class ParquetRowConverter( private[parquet] object ParquetRowConverter { - // just for testing - var tz: TimeZone = DateTimeUtils.TimeZoneGMT - def setTimezone(tz: TimeZone): Unit = { - this.tz = tz - } - def binaryToUnscaledLong(binary: Binary): Long = { // The underlying `ByteBuffer` implementation is guaranteed to be `HeapByteBuffer`, so here // we are using `Binary.toByteBuffer.array()` to steal the underlying byte array without @@ -688,7 +682,7 @@ private[parquet] object ParquetRowConverter { val julianDay = buffer.getInt val utcEpochMicros = DateTimeUtils.fromJulianDay(julianDay, timeOfDayNanos) // avoid expensive time logic if possible - if (tz != DateTimeUtils.TimeZoneGMT) { + if (!DateTimeUtils.isUtcOrGmt(tz)) { // TODO not really sure what the desired behavior here is ... val millis = utcEpochMicros / 1000 val offset = tz.getOffset(millis) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala index b8e7792ad302..76fac7ea3531 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala @@ -173,7 +173,7 @@ private[parquet] class ParquetWriteSupport extends WriteSupport[InternalRow] wit // NOTE: Starting from Spark 1.5, Spark SQL `TimestampType` only has microsecond // precision. Nanosecond parts of timestamp values read from INT96 are simply stripped. val rawMicros = row.getLong(ordinal) - val adjustedMicros = if (timezone == DateTimeUtils.TimeZoneGMT) { + val adjustedMicros = if (DateTimeUtils.isUtcOrGmt(timezone)) { rawMicros } else { // For compatibility with HIVE-12767, always write data in UTC, so adjust *from* the diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala index ab824d4122f9..346757c2047a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala @@ -36,8 +36,6 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.{AttributeMap, AttributeReference, Expression} import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics} import org.apache.spark.sql.execution.FileRelation -import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructField From 223ce2c25b122707c64e4eda77a11bff71fd0cbe Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 2 Feb 2017 10:27:13 -0600 Subject: [PATCH 12/40] more cleanup --- .../parquet/VectorizedColumnReader.java | 1 + .../parquet/ParquetRowConverter.scala | 8 ++-- .../parquet/ParquetWriteSupport.scala | 1 + .../datasources/parquet/ParquetIOSuite.scala | 46 ------------------- .../spark/sql/hive/HiveMetastoreCatalog.scala | 6 --- .../hive/ParquetHiveCompatibilitySuite.scala | 1 - 6 files changed, 5 insertions(+), 58 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java index 58015932dc56..2750275a65d2 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java @@ -99,6 +99,7 @@ public VectorizedColumnReader(ColumnDescriptor descriptor, PageReader pageReader throws IOException { this.descriptor = descriptor; this.pageReader = pageReader; + // If the table has a timezone property, apply the correct conversions. See SPARK-12297. String tzString = conf.get(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY()); if (tzString == null) { tz = DateTimeUtils.TimeZoneGMT(); diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala index 6b67a7058696..b6a2ac4c8f74 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala @@ -19,10 +19,7 @@ package org.apache.spark.sql.execution.datasources.parquet import java.math.{BigDecimal, BigInteger} import java.nio.ByteOrder -import java.sql.Timestamp -import java.util.{Calendar, Date, TimeZone} -import java.util.Formatter.DateTime -import java.util.concurrent.TimeUnit +import java.util.TimeZone import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer @@ -261,6 +258,7 @@ private[parquet] class ParquetRowConverter( case TimestampType => // TODO Implements `TIMESTAMP_MICROS` once parquet-mr has that. + // If the table has a timezone property, apply the correct conversions. See SPARK-12297. val tzString = hadoopConf.get(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY) val tz = if (tzString == null) { DateTimeUtils.TimeZoneGMT @@ -652,7 +650,7 @@ private[parquet] class ParquetRowConverter( private[parquet] object ParquetRowConverter { - def binaryToUnscaledLong(binary: Binary): Long = { + def binaryToUnscaledLong(binary: Binary): Long = { // The underlying `ByteBuffer` implementation is guaranteed to be `HeapByteBuffer`, so here // we are using `Binary.toByteBuffer.array()` to steal the underlying byte array without // copying it. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala index 76fac7ea3531..b07d8ae759a5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala @@ -84,6 +84,7 @@ private[parquet] class ParquetWriteSupport extends WriteSupport[InternalRow] wit configuration.get(SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key).toBoolean } this.rootFieldWriters = schema.map(_.dataType).map(makeWriter) + // If the table has a timezone property, apply the correct conversions. See SPARK-12297. val tzString = configuration.get(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY) timezone = if (tzString == null) { DateTimeUtils.TimeZoneGMT diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala index e2364481be14..5244d386562c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala @@ -755,52 +755,6 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { assert(option.compressionCodecClassName == "UNCOMPRESSED") } } - - test("SPARK-12297: Parquet Timestamp & Hive Timezones") { - // Test that we can correctly adjust parquet timestamps for Hive timezone bug. - withTempPath { dir => - // First, lets generate some parquet data we can use to test this - val schema = StructType(StructField("timestamp", TimestampType) :: Nil) - // intentionally pick a few times right around new years, so time zone will effect many fields - val data = spark.sparkContext.parallelize(Seq( - "2015-12-31 23:50:59.123", - "2015-12-31 22:49:59.123", - "2016-01-01 00:39:59.123", - "2016-01-01 01:29:59.123" - ).map { x => Row(java.sql.Timestamp.valueOf(x)) }) - spark.createDataFrame(data, schema).write.parquet(dir.getCanonicalPath) - - // Ideally, we'd check the parquet schema here, make sure it was int96 - - // now we should try to read that data back. We'll fake a timezone on the table, to see - // what the resulting behavior is - ParquetRowConverter.setTimezone(TimeZone.getTimeZone("PST")) -// spark.conf.set(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, false) - val readInPst = spark.read.parquet(dir.getCanonicalPath) - readInPst.show() - - import functions._ - val originalTsCol = readInPst("timestamp") - val newTable = readInPst.withColumn("year", expr("year(timestamp)")) - assert(newTable.filter("year > '2015'").count() === 0) - - // TODO test: - // * w/ & w/out vectorization - // * filtering - // * partioning - // * DST? - spark.sql( - """CREATE TABLE foobar ( - | year int, - | timestamp timestamp - | ) - | STORED AS PARQUET - """.stripMargin - ) - newTable.createOrReplaceTempView("newTable") - spark.sql("insert into foobar (year, timestamp) select year, timestamp from newTable") - } - } } class JobCommitFailureParquetOutputCommitter(outputPath: Path, context: TaskAttemptContext) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 0c832bd59838..9307572eedce 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -207,8 +207,6 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log val logicalRelation = cached.getOrElse { // We add the timezone to the relation options, which automatically gets injected into // the hadoopConf for the Parquet Converters - logInfo(s"creating HadoopFsRelation from a metastore table with" + - s" ${metastoreRelation.hiveQlTable.getParameters}") val tzKey = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY val tz = Option(metastoreRelation.hiveQlTable.getParameters.get(tzKey)).getOrElse("") val created = @@ -243,8 +241,6 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log } private def convertToParquetRelation(relation: MetastoreRelation): LogicalRelation = { - logInfo(s"creating a parquet relation from a metastore relation on" + - s" ${relation.catalogTable.qualifiedName} : $relation") val defaultSource = new ParquetFileFormat() val fileFormatClass = classOf[ParquetFileFormat] @@ -264,12 +260,10 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log case InsertIntoTable(r: MetastoreRelation, partition, child, overwrite, ifNotExists) // Inserting into partitioned table is not supported in Parquet data source (yet). if !r.hiveQlTable.isPartitioned && shouldConvertMetastoreParquet(r) => - logInfo("checking parquet conversions for insertion") InsertIntoTable(convertToParquetRelation(r), partition, child, overwrite, ifNotExists) // Read path case relation: MetastoreRelation if shouldConvertMetastoreParquet(relation) => - logInfo(s"checking parquet conversions for $plan") val parquetRelation = convertToParquetRelation(relation) SubqueryAlias(relation.tableName, parquetRelation, None) } 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 f73c2fd1cf6c..25062c297510 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 @@ -150,7 +150,6 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi "ARRAY>") } - // TODO GMT / UTC should both use fast path test(s"SPARK-12297: Parquet Timestamp & Hive timezone") { // Check creating parquet tables, writing data into them, and reading it back out under a // variety of conditions: From 5b49ae026044b46f0899a9e792e2b71733c4cb8a Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 2 Feb 2017 12:02:31 -0600 Subject: [PATCH 13/40] fix compatibility --- .../datasources/parquet/VectorizedParquetRecordReader.java | 5 ++--- .../execution/datasources/parquet/ParquetFileFormat.scala | 2 +- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedParquetRecordReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedParquetRecordReader.java index 1a665990f7fa..18c0e0bc1e5a 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedParquetRecordReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedParquetRecordReader.java @@ -106,11 +106,10 @@ public class VectorizedParquetRecordReader extends SpecificParquetRecordReaderBa /** * Implementation of RecordReader API. */ - public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext, - Configuration conf) + public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException, UnsupportedOperationException { super.initialize(inputSplit, taskAttemptContext); - this.conf = conf; + this.conf = taskAttemptContext.getConfiguration(); initializeInternal(); } 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 f6b7495641ab..8a5aeff51bcf 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 @@ -373,7 +373,7 @@ class ParquetFileFormat } val parquetReader = if (enableVectorizedReader) { val vectorizedReader = new VectorizedParquetRecordReader() - vectorizedReader.initialize(split, hadoopAttemptContext, broadcastedHadoopConf.value.value) + vectorizedReader.initialize(split, hadoopAttemptContext) logDebug(s"Appending $partitionSchema ${file.partitionValues}") vectorizedReader.initBatch(partitionSchema, file.partitionValues) if (returningBatch) { From 69b81425c9b68240ddc5411a83ba39ca7d1a74e3 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 2 Mar 2017 21:05:06 -0600 Subject: [PATCH 14/40] wip --- .../sql/catalyst/util/DateTimeUtils.scala | 4 --- .../parquet/VectorizedColumnReader.java | 11 ++++--- .../parquet/ParquetRowConverter.scala | 28 +++++++++------- .../parquet/ParquetWriteSupport.scala | 16 ++++----- .../apache/spark/sql/internal/SQLConf.scala | 2 +- .../hive/ParquetHiveCompatibilitySuite.scala | 33 +++++++------------ 6 files changed, 41 insertions(+), 53 deletions(-) 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 cf3d416770d8..3800ba2bdfb6 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 @@ -482,10 +482,6 @@ object DateTimeUtils { validTimezones.contains(timezoneId) } - def isUtcOrGmt(timeZone: TimeZone): Boolean = { - timeZone.getID == "UTC" || timeZone.getID() == "GMT" - } - /** * Returns the microseconds since year zero (-17999) from microseconds since epoch. */ diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java index 2750275a65d2..533bcc13a1ea 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java @@ -92,7 +92,8 @@ public class VectorizedColumnReader { private final PageReader pageReader; private final ColumnDescriptor descriptor; - private final TimeZone tz; + private final TimeZone storageTz; + private final TimeZone localTz = TimeZone.getDefault(); public VectorizedColumnReader(ColumnDescriptor descriptor, PageReader pageReader, Configuration conf) @@ -102,9 +103,9 @@ public VectorizedColumnReader(ColumnDescriptor descriptor, PageReader pageReader // If the table has a timezone property, apply the correct conversions. See SPARK-12297. String tzString = conf.get(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY()); if (tzString == null) { - tz = DateTimeUtils.TimeZoneGMT(); + storageTz = DateTimeUtils.TimeZoneGMT(); } else { - tz = TimeZone.getTimeZone(tzString); + storageTz = TimeZone.getTimeZone(tzString); } this.maxDefLevel = descriptor.getMaxDefinitionLevel(); @@ -288,7 +289,7 @@ private void decodeDictionaryIds(int rowId, int num, ColumnVector column, // TODO: Convert dictionary of Binaries to dictionary of Longs if (!column.isNullAt(i)) { Binary v = dictionary.decodeToBinary(dictionaryIds.getDictId(i)); - column.putLong(i, ParquetRowConverter.binaryToSQLTimestamp(v, tz)); + column.putLong(i, ParquetRowConverter.binaryToSQLTimestamp(v, storageTz, localTz)); } } } else { @@ -413,7 +414,7 @@ private void readBinaryBatch(int rowId, int num, ColumnVector column) throws IOE if (defColumn.readInteger() == maxDefLevel) { column.putLong(rowId + i, // Read 12 bytes for INT96 - ParquetRowConverter.binaryToSQLTimestamp(data.readBinary(12), tz)); + ParquetRowConverter.binaryToSQLTimestamp(data.readBinary(12), storageTz, localTz)); } else { column.putNull(rowId + i); } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala index b6a2ac4c8f74..fd169a50604d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala @@ -260,16 +260,17 @@ private[parquet] class ParquetRowConverter( // TODO Implements `TIMESTAMP_MICROS` once parquet-mr has that. // If the table has a timezone property, apply the correct conversions. See SPARK-12297. val tzString = hadoopConf.get(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY) - val tz = if (tzString == null) { - DateTimeUtils.TimeZoneGMT + val localTz = TimeZone.getDefault() + val storageTz = if (tzString == null) { + localTz } else { TimeZone.getTimeZone(tzString) } new ParquetPrimitiveConverter(updater) { // Converts nanosecond timestamps stored as INT96 override def addBinary(value: Binary): Unit = { - val timestamp = - ParquetRowConverter.binaryToSQLTimestamp(value, tz) + val timestamp = ParquetRowConverter.binaryToSQLTimestamp(value, storageTz = storageTz, + localTz = localTz) updater.setLong(timestamp) } } @@ -672,19 +673,24 @@ private[parquet] object ParquetRowConverter { unscaled } - def binaryToSQLTimestamp(binary: Binary, tz: TimeZone): SQLTimestamp = { + /** + * Converts an int96 to a SQLTimestamp, given both the storage timezone and the local timezone. + * The timestamp is really meant to be interpreted as a "floating time", but since we + * actually store it as micros since epoch, why we have to apply a conversion when timezones + * change. + * @param binary + * @return + */ + def binaryToSQLTimestamp(binary: Binary, storageTz: TimeZone, localTz: TimeZone): SQLTimestamp = { assert(binary.length() == 12, s"Timestamps (with nanoseconds) are expected to be stored in" + s" 12-byte long binaries. Found a ${binary.length()}-byte binary instead.") val buffer = binary.toByteBuffer.order(ByteOrder.LITTLE_ENDIAN) val timeOfDayNanos = buffer.getLong val julianDay = buffer.getInt val utcEpochMicros = DateTimeUtils.fromJulianDay(julianDay, timeOfDayNanos) - // avoid expensive time logic if possible - if (!DateTimeUtils.isUtcOrGmt(tz)) { - // TODO not really sure what the desired behavior here is ... - val millis = utcEpochMicros / 1000 - val offset = tz.getOffset(millis) - ((millis + offset) * 1000) + (utcEpochMicros % 1000) + // avoid expensive time logic if possible. + if (storageTz.getID() != localTz.getID()) { + DateTimeUtils.convertTz(utcEpochMicros, storageTz, localTz) } else { utcEpochMicros } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala index b07d8ae759a5..8abf88783ecf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala @@ -73,7 +73,8 @@ private[parquet] class ParquetWriteSupport extends WriteSupport[InternalRow] wit // Reusable byte array used to write decimal values private val decimalBuffer = new Array[Byte](minBytesForPrecision(DecimalType.MAX_PRECISION)) - private var timezone: TimeZone = _ + private var storageTz: TimeZone = _ + private var localTz: TimeZone = TimeZone.getDefault() override def init(configuration: Configuration): WriteContext = { val schemaString = configuration.get(ParquetWriteSupport.SPARK_ROW_SCHEMA) @@ -86,8 +87,8 @@ private[parquet] class ParquetWriteSupport extends WriteSupport[InternalRow] wit this.rootFieldWriters = schema.map(_.dataType).map(makeWriter) // If the table has a timezone property, apply the correct conversions. See SPARK-12297. val tzString = configuration.get(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY) - timezone = if (tzString == null) { - DateTimeUtils.TimeZoneGMT + storageTz = if (tzString == null) { + localTz } else { TimeZone.getTimeZone(tzString) } @@ -174,15 +175,10 @@ private[parquet] class ParquetWriteSupport extends WriteSupport[InternalRow] wit // NOTE: Starting from Spark 1.5, Spark SQL `TimestampType` only has microsecond // precision. Nanosecond parts of timestamp values read from INT96 are simply stripped. val rawMicros = row.getLong(ordinal) - val adjustedMicros = if (DateTimeUtils.isUtcOrGmt(timezone)) { + val adjustedMicros = if (localTz.getID() == storageTz.getID()) { rawMicros } else { - // For compatibility with HIVE-12767, always write data in UTC, so adjust *from* the - // table timezone *to* UTC. Eg., if the table was created with TZ = America/Los_Angeles - // (UTC-8 w/out DST), then add 8 hours to the timestamp when we save. - val millisOffset = timezone.getOffset(rawMicros / 1000) - // scalstyle:off - rawMicros - (millisOffset * 1000L) + DateTimeUtils.convertTz(rawMicros, localTz, storageTz) } val (julianDay, timeOfDayNanos) = DateTimeUtils.toJulianDay(adjustedMicros) val buf = ByteBuffer.wrap(timestampBuffer) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 8920ad1c37c4..33ddcff04109 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -675,7 +675,7 @@ object SQLConf { .createWithDefault(TimeZone.getDefault().getID()) val PARQUET_TABLE_INCLUDE_TIMEZONE = - SQLConfigBuilder("spark.sql.session.parquet.timeZone") + buildConf("spark.sql.session.parquet.timeZone") .doc("""Enables inclusion of parquet timezone property in newly created parquet tables""") .booleanConf .createWithDefault(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 25062c297510..543fe21f999b 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,19 +17,16 @@ package org.apache.spark.sql.hive -import java.io.File import java.sql.Timestamp import java.util.TimeZone import org.apache.spark._ -import org.apache.spark.sql.{AnalysisException, Row, SparkSession} +import org.apache.spark.sql.{AnalysisException, Row} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.execution.datasources.parquet.{ParquetCompatibilityTest, ParquetFileFormat} -import org.apache.spark.sql.hive.test.{TestHiveContext, TestHiveSingleton} +import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.internal.StaticSQLConf._ -import org.apache.spark.sql.test.SQLTestUtils -import org.apache.spark.sql.types.{StringType, StructField, StructType, TimestampType} +import org.apache.spark.sql.types.{StructType, TimestampType} class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHiveSingleton { /** @@ -176,7 +173,8 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi val tblProperties = explicitTz.map { tz => raw"""TBLPROPERTIES ($key="$tz")""" }.getOrElse("") - val defaultTz = if (setTableTzByDefault) Some("UTC") else None + val localTz = TimeZone.getDefault.getID() + val defaultTz = if (setTableTzByDefault) Some(localTz) else None spark.sql( raw"""CREATE TABLE $baseTable ( | x int @@ -211,12 +209,14 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi import spark.implicits._ - val rawData = spark.createDataset(Seq( + val originalTsStrings = Seq( "2015-12-31 23:50:59.123", "2015-12-31 22:49:59.123", "2016-01-01 00:39:59.123", "2016-01-01 01:29:59.123" - ).map { x => java.sql.Timestamp.valueOf(x) }) + ) + val rawData = spark.createDataset( + originalTsStrings.map { x => java.sql.Timestamp.valueOf(x) }) // Check writing data out. // We write data into our tables, and then check the raw parquet files to see whether @@ -241,19 +241,8 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi val onDiskLocation = """file:(.*)""".r.findFirstMatchIn(spark.sessionState.catalog .getTableMetadata(TableIdentifier(s"insert_$baseTable")).location).get.group(1) val readFromDisk = spark.read.parquet(onDiskLocation).collect() - .map(_.getAs[Timestamp](0)) - val expectedReadFromDisk = expectedTableTz match { - case Some("America/Los_Angeles") => - // we should take a timestamp that is assumed to be in LA time, and shift it to UTC - // by subtracting the offset from UTC. - val tz = TimeZone.getTimeZone("America/Los_Angeles") - rawData.collect().map { ts => - new Timestamp(ts.getTime - tz.getOffset(ts.getTime)) - } - case _ => - rawData.collect() - } - assert(readFromDisk === expectedReadFromDisk, readFromDisk.mkString(",")) + .map(_.getAs[Timestamp](0).toString()) + assert(readFromDisk === originalTsStrings, readFromDisk.mkString(",")) // check reading data back in // TODO check predicate pushdown From 7ca2c864de3b1a34e2e77e72f9ae51cfada88d65 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 3 Mar 2017 16:33:42 -0600 Subject: [PATCH 15/40] fix --- .../execution/datasources/parquet/VectorizedColumnReader.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java index 533bcc13a1ea..722d77c6e725 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java @@ -103,7 +103,7 @@ public VectorizedColumnReader(ColumnDescriptor descriptor, PageReader pageReader // If the table has a timezone property, apply the correct conversions. See SPARK-12297. String tzString = conf.get(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY()); if (tzString == null) { - storageTz = DateTimeUtils.TimeZoneGMT(); + storageTz = localTz; } else { storageTz = TimeZone.getTimeZone(tzString); } From 6f982d30c7cd4f8ee6e28024c45dcaeaa72bd874 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 6 Mar 2017 13:12:00 -0600 Subject: [PATCH 16/40] fixes; passes tests now --- .../parquet/VectorizedColumnReader.java | 2 +- .../parquet/ParquetWriteSupport.scala | 2 +- .../spark/sql/hive/HiveExternalCatalog.scala | 3 +- .../hive/ParquetHiveCompatibilitySuite.scala | 48 +++++++++++++------ 4 files changed, 38 insertions(+), 17 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java index 722d77c6e725..a10981817f11 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java @@ -102,7 +102,7 @@ public VectorizedColumnReader(ColumnDescriptor descriptor, PageReader pageReader this.pageReader = pageReader; // If the table has a timezone property, apply the correct conversions. See SPARK-12297. String tzString = conf.get(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY()); - if (tzString == null) { + if (tzString == null || tzString == "") { storageTz = localTz; } else { storageTz = TimeZone.getTimeZone(tzString); diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala index 8abf88783ecf..fd7dc6b97031 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala @@ -87,7 +87,7 @@ private[parquet] class ParquetWriteSupport extends WriteSupport[InternalRow] wit this.rootFieldWriters = schema.map(_.dataType).map(makeWriter) // If the table has a timezone property, apply the correct conversions. See SPARK-12297. val tzString = configuration.get(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY) - storageTz = if (tzString == null) { + storageTz = if (tzString == null || tzString == "") { localTz } else { TimeZone.getTimeZone(tzString) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index 956d1ca7081d..b7e00ff48d38 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.hive import java.io.IOException import java.net.URI import java.util +import java.util.TimeZone import scala.collection.mutable import scala.util.control.NonFatal @@ -211,7 +212,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat } val extraTzProp = if (conf.get(SQLConf.PARQUET_TABLE_INCLUDE_TIMEZONE) && tableTz.isEmpty) { - Map(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY -> "UTC") + Map(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY -> TimeZone.getDefault().getID()) } else { Map() } 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 543fe21f999b..f7bc078aaeff 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 @@ -169,12 +169,13 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi withTable(baseTable, s"like_$baseTable", s"select_$baseTable", s"external_$baseTable", s"saveAsTable_$baseTable", s"insert_$baseTable") { withClue(s"table tz default=$setTableTzByDefault; explicitTz = $explicitTz;") { + val localTz = TimeZone.getDefault() + val localTzId = localTz.getID() + val defaultTz = if (setTableTzByDefault) Some(localTzId) else None // check that created tables have correct TBLPROPERTIES val tblProperties = explicitTz.map { tz => raw"""TBLPROPERTIES ($key="$tz")""" }.getOrElse("") - val localTz = TimeZone.getDefault.getID() - val defaultTz = if (setTableTzByDefault) Some(localTz) else None spark.sql( raw"""CREATE TABLE $baseTable ( | x int @@ -237,12 +238,30 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi .map(_.getAs[Timestamp](0)) // no matter what, roundtripping via the table should leave the data unchanged assert(readFromTable === rawData.collect()) - // lets load the raw parquet data on disk, and check if it was adjusted correctly + + // Now we load the raw parquet data on disk, and check if it was adjusted correctly. + // Note that we only store the timezone in the table property, so when we read the + // data this way, we're bypassing all of the conversion logic, and reading the raw + // values in the parquet file. val onDiskLocation = """file:(.*)""".r.findFirstMatchIn(spark.sessionState.catalog .getTableMetadata(TableIdentifier(s"insert_$baseTable")).location).get.group(1) val readFromDisk = spark.read.parquet(onDiskLocation).collect() - .map(_.getAs[Timestamp](0).toString()) - assert(readFromDisk === originalTsStrings, readFromDisk.mkString(",")) + .map(_.getAs[Timestamp](0)) + val expectedReadFromDisk = expectedTableTz match { + case Some(tzId) => + // We should have shifted the data from our local timezone to the storage timezone + // when we saved the data. + val storageTz = TimeZone.getTimeZone(tzId) + rawData.collect().map { ts => + val t = ts.getTime() + new Timestamp(t + storageTz.getOffset(t) - localTz.getOffset(t)) + } + case _ => + rawData.collect() + } + assert(readFromDisk === expectedReadFromDisk, + s"timestamps changed string format after reading back from parquet with " + + s"local = $localTzId & storage = $expectedTableTz") // check reading data back in // TODO check predicate pushdown @@ -266,21 +285,18 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi spark.sql(s"select value from external_$baseTable").collect() .map(_.getAs[Timestamp](0)) val expTimestamps = explicitTz match { - case Some("America/Los_Angeles") => - // this is assuming that the times were adjusted *from* LA *to* UTC when - // they were written out, so on reading we do the reverse. Eg., originally - // the data was 12pm, but it got written out as 8pm. We need to read it - // back as 12pm. The offset is stored as the difference from UTC, so in - // this case it would be -8. - val tz = TimeZone.getTimeZone("America/Los_Angeles") + case Some(tzId) => + val storageTz = TimeZone.getTimeZone(tzId) rawData.collect().map { ts => - new Timestamp(ts.getTime + tz.getOffset(ts.getTime)) + val t = ts.getTime() + new Timestamp(t - storageTz.getOffset(t) + localTz.getOffset(t)) } case _ => // no modification to raw data in parquet rawData.collect() } - assert(collectedFromExternal === expTimestamps) + assert(collectedFromExternal === expTimestamps, + s"collected = ${collectedFromExternal.mkString(",")}") } } } @@ -289,9 +305,13 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi } } checkCreateReadWrite("no_tz", None) + checkCreateReadWrite("local", Some(TimeZone.getDefault().getID())) + // check with a variety of timezones. The unit tests currently are configured to always use + // America/Los_Angeles, but even if they didn't, we'd be sure to cover a non-local timezone. checkCreateReadWrite("UTC", Some("UTC")) checkCreateReadWrite("LA", Some("America/Los_Angeles")) + checkCreateReadWrite("Berlin", Some("Europe/Berlin")) val badTzException = intercept[AnalysisException] { spark.sql( From 2c8a22811f404c751841e9d9f2e8b22780d60f99 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 6 Mar 2017 13:22:11 -0600 Subject: [PATCH 17/40] fix merge --- .../apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 f7bc078aaeff..f0f1ba6a3d4d 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 @@ -243,8 +243,8 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi // Note that we only store the timezone in the table property, so when we read the // data this way, we're bypassing all of the conversion logic, and reading the raw // values in the parquet file. - val onDiskLocation = """file:(.*)""".r.findFirstMatchIn(spark.sessionState.catalog - .getTableMetadata(TableIdentifier(s"insert_$baseTable")).location).get.group(1) + val onDiskLocation = spark.sessionState.catalog + .getTableMetadata(TableIdentifier(s"insert_$baseTable")).location.getPath val readFromDisk = spark.read.parquet(onDiskLocation).collect() .map(_.getAs[Timestamp](0)) val expectedReadFromDisk = expectedTableTz match { From f0b89fdca05d61bff7f79047b5ac72125bfd107e Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 6 Mar 2017 15:59:06 -0600 Subject: [PATCH 18/40] fix --- .../execution/datasources/parquet/VectorizedColumnReader.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java index a10981817f11..9a37a0b57d19 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java @@ -101,7 +101,9 @@ public VectorizedColumnReader(ColumnDescriptor descriptor, PageReader pageReader this.descriptor = descriptor; this.pageReader = pageReader; // If the table has a timezone property, apply the correct conversions. See SPARK-12297. - String tzString = conf.get(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY()); + // The conf is sometimes null in tests. + String tzString = + conf == null ? null : conf.get(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY()); if (tzString == null || tzString == "") { storageTz = localTz; } else { From 46fab8d598b3d852276bacf49d3d19b61372a59a Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 13 Mar 2017 13:18:13 -0500 Subject: [PATCH 19/40] refactor the test --- .../hive/ParquetHiveCompatibilitySuite.scala | 342 ++++++++++-------- 1 file changed, 186 insertions(+), 156 deletions(-) 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 f0f1ba6a3d4d..ddfcf2200ca0 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 @@ -147,184 +147,214 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi "ARRAY>") } - test(s"SPARK-12297: Parquet Timestamp & Hive timezone") { - // Check creating parquet tables, writing data into them, and reading it back out under a - // variety of conditions: - // * global conf for setting table tz by default - // * tables with explicit tz and those without - // * altering table properties directly - // * UTC and non-UTC timezones - Seq(false, true).foreach { setTableTzByDefault => + // Check creating parquet tables, writing data into them, and reading it back out under a + // variety of conditions: + // * global conf for setting table tz by default + // * tables with explicit tz and those without + // * altering table properties directly + // * variety of timezones, local & non-local + Seq(false, true).foreach { setTableTzByDefault => + testCreateReadWriteTablesWithTimezone("no_tz", None, setTableTzByDefault) + val localTz = TimeZone.getDefault.getID() + testCreateReadWriteTablesWithTimezone("local", Some(localTz), setTableTzByDefault) + + // check with a variety of timezones. The unit tests currently are configured to always use + // America/Los_Angeles, but even if they didn't, we'd be sure to cover a non-local timezone. + Seq( + "UTC" -> "UTC", + "LA" -> "America/Los_Angeles", + "Berlin" -> "Europe/Berlin" + ).foreach { case (tableName, zone) => + if (zone != localTz) { + testCreateReadWriteTablesWithTimezone(tableName, Some(zone), setTableTzByDefault) + } + } + } + + private def checkHasTz(table: String, tz: Option[String]): Unit = { + val tableMetadata = spark.sessionState.catalog.getTableMetadata(TableIdentifier(table)) + val key = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY + assert(tableMetadata.properties.get(key) === tz) + } + + private def testCreateReadWriteTablesWithTimezone( + baseTable: String, + explicitTz: Option[String], + setTableTzByDefault: Boolean): Unit = { + val key = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY + test(s"SPARK-12297: Parquet Timestamp & Hive timezone; " + + s"setTzByDefault = $setTableTzByDefault; explicitTz = $explicitTz") { // we're cheating a bit here, in general SparkConf isn't meant to be set at runtime, // but its OK in this case, and lets us run this test, because these tests don't like // creating multiple HiveContexts in the same jvm sparkContext.conf.set( SQLConf.PARQUET_TABLE_INCLUDE_TIMEZONE.key, setTableTzByDefault.toString) - val key = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY - def checkHasTz(table: String, tz: Option[String]): Unit = { - val tableMetadata = spark.sessionState.catalog.getTableMetadata(TableIdentifier(table)) - assert(tableMetadata.properties.get(key) === tz) - } - def checkCreateReadWrite(baseTable: String, explicitTz: Option[String]): Unit = { - withTable(baseTable, s"like_$baseTable", s"select_$baseTable", s"external_$baseTable", - s"saveAsTable_$baseTable", s"insert_$baseTable") { - withClue(s"table tz default=$setTableTzByDefault; explicitTz = $explicitTz;") { - val localTz = TimeZone.getDefault() - val localTzId = localTz.getID() - val defaultTz = if (setTableTzByDefault) Some(localTzId) else None - // check that created tables have correct TBLPROPERTIES - val tblProperties = explicitTz.map { - tz => raw"""TBLPROPERTIES ($key="$tz")""" - }.getOrElse("") - spark.sql( - raw"""CREATE TABLE $baseTable ( - | x int - | ) - | STORED AS PARQUET - | $tblProperties + + withTable(baseTable, s"like_$baseTable", s"select_$baseTable", s"external_$baseTable", + s"saveAsTable_$baseTable", s"insert_$baseTable") { + val localTz = TimeZone.getDefault() + val localTzId = localTz.getID() + val defaultTz = if (setTableTzByDefault) Some(localTzId) else None + // check that created tables have correct TBLPROPERTIES + val tblProperties = explicitTz.map { + tz => raw"""TBLPROPERTIES ($key="$tz")""" + }.getOrElse("") + spark.sql( + raw"""CREATE TABLE $baseTable ( + | x int + | ) + | STORED AS PARQUET + | $tblProperties """.stripMargin) - val expectedTableTz = explicitTz.orElse(defaultTz) - checkHasTz(baseTable, expectedTableTz) - spark.sql(s"CREATE TABLE like_$baseTable LIKE $baseTable") - checkHasTz(s"like_$baseTable", expectedTableTz) - spark.sql( - raw"""CREATE TABLE select_$baseTable - | STORED AS PARQUET - | AS - | SELECT * from $baseTable + val expectedTableTz = explicitTz.orElse(defaultTz) + checkHasTz(baseTable, expectedTableTz) + spark.sql(s"CREATE TABLE like_$baseTable LIKE $baseTable") + checkHasTz(s"like_$baseTable", expectedTableTz) + spark.sql( + raw"""CREATE TABLE select_$baseTable + + + | STORED AS PARQUET + | AS + | SELECT * from $baseTable """.stripMargin) - checkHasTz(s"select_$baseTable", defaultTz) - - // check alter table, setting, unsetting, resetting the property - spark.sql( - raw"""ALTER TABLE $baseTable SET TBLPROPERTIES ($key="America/Los_Angeles")""") - checkHasTz(baseTable, Some("America/Los_Angeles")) - spark.sql(raw"""ALTER TABLE $baseTable SET TBLPROPERTIES ($key="UTC")""") - checkHasTz(baseTable, Some("UTC")) - spark.sql(raw"""ALTER TABLE $baseTable UNSET TBLPROPERTIES ($key)""") - checkHasTz(baseTable, None) - explicitTz.foreach { tz => - spark.sql( raw"""ALTER TABLE $baseTable SET TBLPROPERTIES ($key="$tz")""") - checkHasTz(baseTable, expectedTableTz) - } + checkHasTz(s"select_$baseTable", defaultTz) + + // check alter table, setting, unsetting, resetting the property + spark.sql( + raw"""ALTER TABLE $baseTable SET TBLPROPERTIES ($key="America/Los_Angeles")""") + checkHasTz(baseTable, Some("America/Los_Angeles")) + spark.sql(raw"""ALTER TABLE $baseTable SET TBLPROPERTIES ($key="UTC")""") + checkHasTz(baseTable, Some("UTC")) + spark.sql(raw"""ALTER TABLE $baseTable UNSET TBLPROPERTIES ($key)""") + checkHasTz(baseTable, None) + explicitTz.foreach { tz => + spark.sql( raw"""ALTER TABLE $baseTable SET TBLPROPERTIES ($key="$tz")""") + checkHasTz(baseTable, expectedTableTz) + } - import spark.implicits._ - val originalTsStrings = Seq( - "2015-12-31 23:50:59.123", - "2015-12-31 22:49:59.123", - "2016-01-01 00:39:59.123", - "2016-01-01 01:29:59.123" - ) - val rawData = spark.createDataset( - originalTsStrings.map { x => java.sql.Timestamp.valueOf(x) }) - - // Check writing data out. - // We write data into our tables, and then check the raw parquet files to see whether - // the correct conversion was applied. - rawData.write.saveAsTable(s"saveAsTable_$baseTable") - checkHasTz(s"saveAsTable_$baseTable", defaultTz) - spark.sql( - raw"""CREATE TABLE insert_$baseTable ( - | ts timestamp - | ) - | STORED AS PARQUET - | $tblProperties + import spark.implicits._ + val originalTsStrings = Seq( + "2015-12-31 23:50:59.123", + "2015-12-31 22:49:59.123", + "2016-01-01 00:39:59.123", + "2016-01-01 01:29:59.123" + ) + val rawData = spark.createDataset( + originalTsStrings.map { x => java.sql.Timestamp.valueOf(x) }) + + // Check writing data out. + // We write data into our tables, and then check the raw parquet files to see whether + // the correct conversion was applied. + rawData.write.saveAsTable(s"saveAsTable_$baseTable") + checkHasTz(s"saveAsTable_$baseTable", defaultTz) + spark.sql( + raw"""CREATE TABLE insert_$baseTable ( + | ts timestamp + | ) + | STORED AS PARQUET + | $tblProperties """.stripMargin) - checkHasTz(s"insert_$baseTable", expectedTableTz) - rawData.createOrReplaceTempView(s"tempView_$baseTable") - spark.sql(s"INSERT INTO insert_$baseTable SELECT value AS ts FROM tempView_$baseTable") - val readFromTable = spark.table(s"insert_$baseTable").collect() - .map(_.getAs[Timestamp](0)) - // no matter what, roundtripping via the table should leave the data unchanged - assert(readFromTable === rawData.collect()) - - // Now we load the raw parquet data on disk, and check if it was adjusted correctly. - // Note that we only store the timezone in the table property, so when we read the - // data this way, we're bypassing all of the conversion logic, and reading the raw - // values in the parquet file. - val onDiskLocation = spark.sessionState.catalog - .getTableMetadata(TableIdentifier(s"insert_$baseTable")).location.getPath - val readFromDisk = spark.read.parquet(onDiskLocation).collect() - .map(_.getAs[Timestamp](0)) - val expectedReadFromDisk = expectedTableTz match { - case Some(tzId) => - // We should have shifted the data from our local timezone to the storage timezone - // when we saved the data. - val storageTz = TimeZone.getTimeZone(tzId) - rawData.collect().map { ts => - val t = ts.getTime() - new Timestamp(t + storageTz.getOffset(t) - localTz.getOffset(t)) - } - case _ => - rawData.collect() + checkHasTz(s"insert_$baseTable", expectedTableTz) + rawData.createOrReplaceTempView(s"tempView_$baseTable") + spark.sql(s"INSERT INTO insert_$baseTable SELECT value AS ts FROM tempView_$baseTable") + val readFromTable = spark.table(s"insert_$baseTable").collect() + .map(_.getAs[Timestamp](0)) + // no matter what, roundtripping via the table should leave the data unchanged + assert(readFromTable === rawData.collect()) + + // Now we load the raw parquet data on disk, and check if it was adjusted correctly. + // Note that we only store the timezone in the table property, so when we read the + // data this way, we're bypassing all of the conversion logic, and reading the raw + // values in the parquet file. + val onDiskLocation = spark.sessionState.catalog + .getTableMetadata(TableIdentifier(s"insert_$baseTable")).location.getPath + val readFromDisk = spark.read.parquet(onDiskLocation).collect() + .map(_.getAs[Timestamp](0)) + val expectedReadFromDisk = expectedTableTz match { + case Some(tzId) => + // We should have shifted the data from our local timezone to the storage timezone + // when we saved the data. + val storageTz = TimeZone.getTimeZone(tzId) + rawData.collect().map { ts => + val t = ts.getTime() + new Timestamp(t + storageTz.getOffset(t) - localTz.getOffset(t)) } - assert(readFromDisk === expectedReadFromDisk, - s"timestamps changed string format after reading back from parquet with " + - s"local = $localTzId & storage = $expectedTableTz") - - // check reading data back in - // TODO check predicate pushdown - // we intentionally save this data directly, without creating a table, so we can - // see that the data is read back differently depending on table properties - withTempPath { path => - rawData.write.parquet(path.getCanonicalPath) - val options = Map("path" -> path.getCanonicalPath) ++ - explicitTz.map { tz => Map(key -> tz) }.getOrElse(Map()) - - spark.catalog.createTable( - tableName = s"external_$baseTable", - source = "parquet", - schema = new StructType().add("value", TimestampType), - options = options - ) - Seq(false, true).foreach { vectorized => - withSQLConf((SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, vectorized.toString)) { - withClue(s"vectorized = $vectorized;") { - val collectedFromExternal = - spark.sql(s"select value from external_$baseTable").collect() - .map(_.getAs[Timestamp](0)) - val expTimestamps = explicitTz match { - case Some(tzId) => - val storageTz = TimeZone.getTimeZone(tzId) - rawData.collect().map { ts => - val t = ts.getTime() - new Timestamp(t - storageTz.getOffset(t) + localTz.getOffset(t)) - } - case _ => - // no modification to raw data in parquet - rawData.collect() + case _ => + rawData.collect() + } + assert(readFromDisk === expectedReadFromDisk, + s"timestamps changed string format after reading back from parquet with " + + s"local = $localTzId & storage = $expectedTableTz") + + // check reading data back in + // TODO check predicate pushdown + // we intentionally save this data directly, without creating a table, so we can + // see that the data is read back differently depending on table properties + withTempPath { path => + rawData.write.parquet(path.getCanonicalPath) + val options = Map("path" -> path.getCanonicalPath) ++ + explicitTz.map { tz => Map(key -> tz) }.getOrElse(Map()) + + spark.catalog.createTable( + tableName = s"external_$baseTable", + source = "parquet", + schema = new StructType().add("value", TimestampType), + options = options + ) + Seq(false, true).foreach { vectorized => + withSQLConf((SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, vectorized.toString)) { + withClue(s"vectorized = $vectorized;") { + val collectedFromExternal = + spark.sql(s"select value from external_$baseTable").collect() + .map(_.getAs[Timestamp](0)) + val expTimestamps = explicitTz match { + case Some(tzId) => + val storageTz = TimeZone.getTimeZone(tzId) + rawData.collect().map { ts => + val t = ts.getTime() + new Timestamp(t - storageTz.getOffset(t) + localTz.getOffset(t)) } - assert(collectedFromExternal === expTimestamps, - s"collected = ${collectedFromExternal.mkString(",")}") - } + case _ => + // no modification to raw data in parquet + rawData.collect() } + assert(collectedFromExternal === expTimestamps, + s"collected = ${collectedFromExternal.mkString(",")}") } } } } } - checkCreateReadWrite("no_tz", None) - checkCreateReadWrite("local", Some(TimeZone.getDefault().getID())) + } + } - // check with a variety of timezones. The unit tests currently are configured to always use - // America/Los_Angeles, but even if they didn't, we'd be sure to cover a non-local timezone. - checkCreateReadWrite("UTC", Some("UTC")) - checkCreateReadWrite("LA", Some("America/Los_Angeles")) - checkCreateReadWrite("Berlin", Some("Europe/Berlin")) + test("SPARK-12297: join after change in timezone") { + import spark.implicits._ + val originalTsStrings = Seq( + "2015-12-31 23:50:59.123", + "2015-12-31 22:49:59.123", + "2016-01-01 00:39:59.123", + "2016-01-01 01:29:59.123" + ) + val rawData = spark.createDataset( + originalTsStrings.map { x => java.sql.Timestamp.valueOf(x) }) - val badTzException = intercept[AnalysisException] { - spark.sql( - raw"""CREATE TABLE bad_tz_table ( - | x int - | ) - | STORED AS PARQUET - | TBLPROPERTIES ($key="Blart Versenwald III") - """.stripMargin) - } - assert(badTzException.getMessage.contains("Blart Versenwald III")) + } + test("SPARK-12297: exception on bad timezone") { + val key = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY + val badTzException = intercept[AnalysisException] { + spark.sql( + raw"""CREATE TABLE bad_tz_table ( + | x int + | ) + | STORED AS PARQUET + | TBLPROPERTIES ($key="Blart Versenwald III") + """.stripMargin) } + assert(badTzException.getMessage.contains("Blart Versenwald III")) } } From c242fb89e2861c81c44877a3acfded9073b9104b Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 13 Mar 2017 13:23:13 -0500 Subject: [PATCH 20/40] cleanup --- .../parquet/VectorizedColumnReader.java | 2 +- .../sql/hive/ParquetHiveCompatibilitySuite.scala | 14 -------------- 2 files changed, 1 insertion(+), 15 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java index 9a37a0b57d19..7591532c52b0 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java @@ -104,7 +104,7 @@ public VectorizedColumnReader(ColumnDescriptor descriptor, PageReader pageReader // The conf is sometimes null in tests. String tzString = conf == null ? null : conf.get(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY()); - if (tzString == null || tzString == "") { + if (tzString == null || tzString.isEmpty()) { storageTz = localTz; } else { storageTz = TimeZone.getTimeZone(tzString); 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 ddfcf2200ca0..8d7755a3288e 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 @@ -330,19 +330,6 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi } } - test("SPARK-12297: join after change in timezone") { - import spark.implicits._ - val originalTsStrings = Seq( - "2015-12-31 23:50:59.123", - "2015-12-31 22:49:59.123", - "2016-01-01 00:39:59.123", - "2016-01-01 01:29:59.123" - ) - val rawData = spark.createDataset( - originalTsStrings.map { x => java.sql.Timestamp.valueOf(x) }) - - } - test("SPARK-12297: exception on bad timezone") { val key = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY val badTzException = intercept[AnalysisException] { @@ -356,5 +343,4 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi } assert(badTzException.getMessage.contains("Blart Versenwald III")) } - } From c87a57323ac1264ed4528a8cd2e26d342ca117f6 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 14 Mar 2017 11:29:15 -0500 Subject: [PATCH 21/40] reset timezone property after tests; make tests more granular --- .../hive/ParquetHiveCompatibilitySuite.scala | 78 +++++++++++++------ 1 file changed, 56 insertions(+), 22 deletions(-) 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 8d7755a3288e..534406a0dd6c 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 @@ -20,7 +20,8 @@ package org.apache.spark.sql.hive import java.sql.Timestamp import java.util.TimeZone -import org.apache.spark._ +import org.scalatest.BeforeAndAfterEach + import org.apache.spark.sql.{AnalysisException, Row} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.execution.datasources.parquet.{ParquetCompatibilityTest, ParquetFileFormat} @@ -28,7 +29,8 @@ import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{StructType, TimestampType} -class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHiveSingleton { +class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHiveSingleton + with BeforeAndAfterEach { /** * Set the staging directory (and hence path to ignore Parquet files under) * to the default value of hive.exec.stagingdir. @@ -46,6 +48,17 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi """.stripMargin) } + override def afterEach(): Unit = { + sparkContext.conf.set( + SQLConf.PARQUET_TABLE_INCLUDE_TIMEZONE.key, "false") + try { + // drop all databases, tables and functions after each test + spark.sessionState.catalog.reset() + } finally { + super.afterEach() + } + } + private def testParquetHiveCompatibility(row: Row, hiveTypes: String*): Unit = { withTable("parquet_compat") { withTempPath { dir => @@ -154,10 +167,11 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi // * altering table properties directly // * variety of timezones, local & non-local Seq(false, true).foreach { setTableTzByDefault => - testCreateReadWriteTablesWithTimezone("no_tz", None, setTableTzByDefault) + testCreateAlterTablesWithTimezone("no_tz", None, setTableTzByDefault) + testReadWriteTablesWithTimezone("no_tz", None, setTableTzByDefault) val localTz = TimeZone.getDefault.getID() - testCreateReadWriteTablesWithTimezone("local", Some(localTz), setTableTzByDefault) - + testCreateAlterTablesWithTimezone("local", Some(localTz), setTableTzByDefault) + testReadWriteTablesWithTimezone("local", Some(localTz), setTableTzByDefault) // check with a variety of timezones. The unit tests currently are configured to always use // America/Los_Angeles, but even if they didn't, we'd be sure to cover a non-local timezone. Seq( @@ -166,32 +180,30 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi "Berlin" -> "Europe/Berlin" ).foreach { case (tableName, zone) => if (zone != localTz) { - testCreateReadWriteTablesWithTimezone(tableName, Some(zone), setTableTzByDefault) + testCreateAlterTablesWithTimezone(tableName, Some(zone), setTableTzByDefault) + testReadWriteTablesWithTimezone(tableName, Some(zone), setTableTzByDefault) } } } private def checkHasTz(table: String, tz: Option[String]): Unit = { val tableMetadata = spark.sessionState.catalog.getTableMetadata(TableIdentifier(table)) - val key = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY - assert(tableMetadata.properties.get(key) === tz) + assert(tableMetadata.properties.get(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY) === tz) } - private def testCreateReadWriteTablesWithTimezone( + private def testCreateAlterTablesWithTimezone( baseTable: String, explicitTz: Option[String], setTableTzByDefault: Boolean): Unit = { - val key = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY - test(s"SPARK-12297: Parquet Timestamp & Hive timezone; " + + test(s"SPARK-12297: Create and Alter Parquet tables and timezones; " + s"setTzByDefault = $setTableTzByDefault; explicitTz = $explicitTz") { // we're cheating a bit here, in general SparkConf isn't meant to be set at runtime, // but its OK in this case, and lets us run this test, because these tests don't like // creating multiple HiveContexts in the same jvm - sparkContext.conf.set( - SQLConf.PARQUET_TABLE_INCLUDE_TIMEZONE.key, setTableTzByDefault.toString) - - withTable(baseTable, s"like_$baseTable", s"select_$baseTable", s"external_$baseTable", - s"saveAsTable_$baseTable", s"insert_$baseTable") { + val key = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY + sparkContext.conf.set(SQLConf.PARQUET_TABLE_INCLUDE_TIMEZONE.key, + setTableTzByDefault.toString) + withTable(baseTable, s"like_$baseTable", s"select_$baseTable") { val localTz = TimeZone.getDefault() val localTzId = localTz.getID() val defaultTz = if (setTableTzByDefault) Some(localTzId) else None @@ -212,8 +224,6 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi checkHasTz(s"like_$baseTable", expectedTableTz) spark.sql( raw"""CREATE TABLE select_$baseTable - - | STORED AS PARQUET | AS | SELECT * from $baseTable @@ -224,15 +234,40 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi spark.sql( raw"""ALTER TABLE $baseTable SET TBLPROPERTIES ($key="America/Los_Angeles")""") checkHasTz(baseTable, Some("America/Los_Angeles")) - spark.sql(raw"""ALTER TABLE $baseTable SET TBLPROPERTIES ($key="UTC")""") + spark.sql( raw"""ALTER TABLE $baseTable SET TBLPROPERTIES ($key="UTC")""") checkHasTz(baseTable, Some("UTC")) - spark.sql(raw"""ALTER TABLE $baseTable UNSET TBLPROPERTIES ($key)""") + spark.sql( raw"""ALTER TABLE $baseTable UNSET TBLPROPERTIES ($key)""") checkHasTz(baseTable, None) explicitTz.foreach { tz => spark.sql( raw"""ALTER TABLE $baseTable SET TBLPROPERTIES ($key="$tz")""") checkHasTz(baseTable, expectedTableTz) } + } + } + } + + private def testReadWriteTablesWithTimezone( + baseTable: String, + explicitTz: Option[String], + setTableTzByDefault: Boolean): Unit = { + val key = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY + test(s"SPARK-12297: Read And Write from Parquet tables with Timestamps; " + + s"setTzByDefault = $setTableTzByDefault; explicitTz = $explicitTz") { + // we're cheating a bit here, in general SparkConf isn't meant to be set at runtime, + // but its OK in this case, and lets us run this test, because these tests don't like + // creating multiple HiveContexts in the same jvm + sparkContext.conf.set( + SQLConf.PARQUET_TABLE_INCLUDE_TIMEZONE.key, setTableTzByDefault.toString) + withTable(s"external_$baseTable", s"saveAsTable_$baseTable", s"insert_$baseTable") { + val localTz = TimeZone.getDefault() + val localTzId = localTz.getID() + val defaultTz = if (setTableTzByDefault) Some(localTzId) else None + val expectedTableTz = explicitTz.orElse(defaultTz) + // check that created tables have correct TBLPROPERTIES + val tblProperties = explicitTz.map { + tz => raw"""TBLPROPERTIES ($key="$tz")""" + }.getOrElse("") import spark.implicits._ val originalTsStrings = Seq( @@ -257,8 +292,7 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi | $tblProperties """.stripMargin) checkHasTz(s"insert_$baseTable", expectedTableTz) - rawData.createOrReplaceTempView(s"tempView_$baseTable") - spark.sql(s"INSERT INTO insert_$baseTable SELECT value AS ts FROM tempView_$baseTable") + rawData.write.insertInto(s"insert_$baseTable") val readFromTable = spark.table(s"insert_$baseTable").collect() .map(_.getAs[Timestamp](0)) // no matter what, roundtripping via the table should leave the data unchanged From f4dca27e301c1ce94e7175c6a852a4e94cea0555 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 15 Mar 2017 11:50:39 -0500 Subject: [PATCH 22/40] separate tests for reading & writing --- .../hive/ParquetHiveCompatibilitySuite.scala | 67 ++++++++++++------- 1 file changed, 44 insertions(+), 23 deletions(-) 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 534406a0dd6c..7f4d686cfeac 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 @@ -22,7 +22,7 @@ import java.util.TimeZone import org.scalatest.BeforeAndAfterEach -import org.apache.spark.sql.{AnalysisException, Row} +import org.apache.spark.sql.{AnalysisException, Dataset, Row} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.execution.datasources.parquet.{ParquetCompatibilityTest, ParquetFileFormat} import org.apache.spark.sql.hive.test.TestHiveSingleton @@ -167,11 +167,9 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi // * altering table properties directly // * variety of timezones, local & non-local Seq(false, true).foreach { setTableTzByDefault => - testCreateAlterTablesWithTimezone("no_tz", None, setTableTzByDefault) - testReadWriteTablesWithTimezone("no_tz", None, setTableTzByDefault) + testCreateWriteRead("no_tz", None, setTableTzByDefault) val localTz = TimeZone.getDefault.getID() - testCreateAlterTablesWithTimezone("local", Some(localTz), setTableTzByDefault) - testReadWriteTablesWithTimezone("local", Some(localTz), setTableTzByDefault) + testCreateWriteRead("local", Some(localTz), setTableTzByDefault) // check with a variety of timezones. The unit tests currently are configured to always use // America/Los_Angeles, but even if they didn't, we'd be sure to cover a non-local timezone. Seq( @@ -180,12 +178,20 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi "Berlin" -> "Europe/Berlin" ).foreach { case (tableName, zone) => if (zone != localTz) { - testCreateAlterTablesWithTimezone(tableName, Some(zone), setTableTzByDefault) - testReadWriteTablesWithTimezone(tableName, Some(zone), setTableTzByDefault) + testCreateWriteRead(tableName, Some(zone), setTableTzByDefault) } } } + private def testCreateWriteRead( + baseTable: String, + explicitTz: Option[String], + setTableTzByDefault: Boolean): Unit = { + testCreateAlterTablesWithTimezone(baseTable, explicitTz, setTableTzByDefault) + testWriteTablesWithTimezone(baseTable, explicitTz, setTableTzByDefault) + testReadTablesWithTimezone(baseTable, explicitTz, setTableTzByDefault) + } + private def checkHasTz(table: String, tz: Option[String]): Unit = { val tableMetadata = spark.sessionState.catalog.getTableMetadata(TableIdentifier(table)) assert(tableMetadata.properties.get(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY) === tz) @@ -246,20 +252,29 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi } } - private def testReadWriteTablesWithTimezone( + private def createRawData(): Dataset[Timestamp] = { + import spark.implicits._ + val originalTsStrings = Seq( + "2015-12-31 23:50:59.123", + "2015-12-31 22:49:59.123", + "2016-01-01 00:39:59.123", + "2016-01-01 01:29:59.123" + ) + spark.createDataset( + originalTsStrings.map { x => java.sql.Timestamp.valueOf(x) }) + } + + private def testWriteTablesWithTimezone( baseTable: String, explicitTz: Option[String], setTableTzByDefault: Boolean): Unit = { val key = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY - test(s"SPARK-12297: Read And Write from Parquet tables with Timestamps; " + + test(s"SPARK-12297: Write to Parquet tables with Timestamps; " + s"setTzByDefault = $setTableTzByDefault; explicitTz = $explicitTz") { - // we're cheating a bit here, in general SparkConf isn't meant to be set at runtime, - // but its OK in this case, and lets us run this test, because these tests don't like - // creating multiple HiveContexts in the same jvm sparkContext.conf.set( SQLConf.PARQUET_TABLE_INCLUDE_TIMEZONE.key, setTableTzByDefault.toString) - withTable(s"external_$baseTable", s"saveAsTable_$baseTable", s"insert_$baseTable") { + withTable(s"saveAsTable_$baseTable", s"insert_$baseTable") { val localTz = TimeZone.getDefault() val localTzId = localTz.getID() val defaultTz = if (setTableTzByDefault) Some(localTzId) else None @@ -269,16 +284,8 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi tz => raw"""TBLPROPERTIES ($key="$tz")""" }.getOrElse("") - import spark.implicits._ - val originalTsStrings = Seq( - "2015-12-31 23:50:59.123", - "2015-12-31 22:49:59.123", - "2016-01-01 00:39:59.123", - "2016-01-01 01:29:59.123" - ) - val rawData = spark.createDataset( - originalTsStrings.map { x => java.sql.Timestamp.valueOf(x) }) + val rawData = createRawData() // Check writing data out. // We write data into our tables, and then check the raw parquet files to see whether // the correct conversion was applied. @@ -321,11 +328,25 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi assert(readFromDisk === expectedReadFromDisk, s"timestamps changed string format after reading back from parquet with " + s"local = $localTzId & storage = $expectedTableTz") + } + } + } - // check reading data back in + private def testReadTablesWithTimezone( + baseTable: String, + explicitTz: Option[String], + setTableTzByDefault: Boolean): Unit = { + val key = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY + test(s"SPARK-12297: Read from Parquet tables with Timestamps; " + + s"setTzByDefault = $setTableTzByDefault; explicitTz = $explicitTz") { + sparkContext.conf.set( + SQLConf.PARQUET_TABLE_INCLUDE_TIMEZONE.key, setTableTzByDefault.toString) + withTable(s"external_$baseTable") { // TODO check predicate pushdown // we intentionally save this data directly, without creating a table, so we can // see that the data is read back differently depending on table properties + val localTz = TimeZone.getDefault() + val rawData = createRawData() withTempPath { path => rawData.write.parquet(path.getCanonicalPath) val options = Map("path" -> path.getCanonicalPath) ++ From d951443d7dddd187fe119f01cb9ee16459f4a346 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 15 Mar 2017 12:08:35 -0500 Subject: [PATCH 23/40] fix merge --- .../scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index e17ec55a2226..0c3cf5089186 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -39,7 +39,7 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.escapePathName import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.ColumnStat -import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.PartitioningUtils import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat From 39f506c81187b466809ebca21b7d1bc4c1e82f7a Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 23 Mar 2017 10:34:33 -0500 Subject: [PATCH 24/40] cleanup --- .../sql/execution/datasources/parquet/ParquetIOSuite.scala | 2 -- .../spark/sql/hive/ParquetHiveCompatibilitySuite.scala | 5 ++--- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala index 5244d386562c..dbdcd230a4de 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.execution.datasources.parquet -import java.util.TimeZone - import scala.collection.JavaConverters._ import scala.collection.mutable import scala.reflect.ClassTag 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 7f4d686cfeac..6b6a2b49d163 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 @@ -154,14 +154,13 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi } test("SPARK-16344: array of struct with a single field named 'array_element'") { - testParquetHiveCompatibility( Row(Seq(Row(1))), "ARRAY>") } - // Check creating parquet tables, writing data into them, and reading it back out under a - // variety of conditions: + // Check creating parquet tables with timestamps, writing data into them, and reading it back out + // under a variety of conditions: // * global conf for setting table tz by default // * tables with explicit tz and those without // * altering table properties directly From 17565e89d5858fdd9a7de857c0a73f960cae9ae6 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 24 Mar 2017 15:16:56 -0500 Subject: [PATCH 25/40] remove config for setting table time zone automatically --- .../sql/catalyst/util/DateTimeUtils.scala | 5 -- .../apache/spark/sql/internal/SQLConf.scala | 6 -- .../VectorizedParquetRecordReader.java | 1 + .../spark/sql/hive/HiveExternalCatalog.scala | 43 ++++-------- .../hive/ParquetHiveCompatibilitySuite.scala | 69 ++++++++----------- 5 files changed, 40 insertions(+), 84 deletions(-) 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 197ef77d1550..9b94c1e2b40b 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 @@ -479,11 +479,6 @@ object DateTimeUtils { false } - lazy val validTimezones = TimeZone.getAvailableIDs().toSet - def isValidTimezone(timezoneId: String): Boolean = { - validTimezones.contains(timezoneId) - } - /** * Returns the microseconds since year zero (-17999) from microseconds since epoch. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index f32270925b57..d5006c16469b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -738,12 +738,6 @@ object SQLConf { .stringConf .createWithDefault(TimeZone.getDefault().getID()) - val PARQUET_TABLE_INCLUDE_TIMEZONE = - buildConf("spark.sql.session.parquet.timeZone") - .doc("""Enables inclusion of parquet timezone property in newly created parquet tables""") - .booleanConf - .createWithDefault(false) - val WINDOW_EXEC_BUFFER_SPILL_THRESHOLD = buildConf("spark.sql.windowExec.buffer.spill.threshold") .internal() diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedParquetRecordReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedParquetRecordReader.java index 18c0e0bc1e5a..d8974ddf2470 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedParquetRecordReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedParquetRecordReader.java @@ -106,6 +106,7 @@ public class VectorizedParquetRecordReader extends SpecificParquetRecordReaderBa /** * Implementation of RecordReader API. */ + @Override public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException, UnsupportedOperationException { super.initialize(inputSplit, taskAttemptContext); diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index 0c3cf5089186..33b21be37203 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.hive import java.io.IOException import java.lang.reflect.InvocationTargetException import java.util -import java.util.TimeZone import scala.collection.mutable import scala.util.control.NonFatal @@ -39,12 +38,11 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.escapePathName import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.ColumnStat -import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.PartitioningUtils -import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.hive.client.HiveClient -import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} +import org.apache.spark.sql.internal.HiveSerDe import org.apache.spark.sql.internal.StaticSQLConf._ import org.apache.spark.sql.types.{DataType, StructType} @@ -208,54 +206,37 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat throw new TableAlreadyExistsException(db = db, table = table) } - // add the extra tz property only if configured to do so, and the table does not include it - // explicitly - val tableTz = tableDefinition.properties.get(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY) - tableTz.foreach { tz => - if (!DateTimeUtils.isValidTimezone(tz)) { - throw new AnalysisException(s"Cannot set" + - s" ${ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY} to invalid timezone $tz") - } - } - val extraTzProp = - if (conf.get(SQLConf.PARQUET_TABLE_INCLUDE_TIMEZONE) && tableTz.isEmpty) { - Map(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY -> TimeZone.getDefault().getID()) - } else { - Map() - } - val tableDef = tableDefinition.copy(properties = tableDefinition.properties ++ extraTzProp) - if (tableDef.tableType == VIEW) { - client.createTable(tableDef, ignoreIfExists) + if (tableDefinition.tableType == VIEW) { + client.createTable(tableDefinition, ignoreIfExists) } else { // Ideally we should not create a managed table with location, but Hive serde table can // specify location for managed table. And in [[CreateDataSourceTableAsSelectCommand]] we have // to create the table directory and write out data before we create this table, to avoid // exposing a partial written table. - val needDefaultTableLocation = tableDef.tableType == MANAGED && - tableDef.storage.locationUri.isEmpty + val needDefaultTableLocation = tableDefinition.tableType == MANAGED && + tableDefinition.storage.locationUri.isEmpty val tableLocation = if (needDefaultTableLocation) { Some(CatalogUtils.stringToURI(defaultTablePath(tableDefinition.identifier))) } else { - tableDef.storage.locationUri + tableDefinition.storage.locationUri } - if (DDLUtils.isHiveTable(tableDef)) { - - val tableWithDataSourceProps = tableDef.copy( + if (DDLUtils.isHiveTable(tableDefinition)) { + val tableWithDataSourceProps = tableDefinition.copy( // We can't leave `locationUri` empty and count on Hive metastore to set a default table // location, because Hive metastore uses hive.metastore.warehouse.dir to generate default // table location for tables in default database, while we expect to use the location of // default database. - storage = tableDef.storage.copy(locationUri = tableLocation), + storage = tableDefinition.storage.copy(locationUri = tableLocation), // Here we follow data source tables and put table metadata like table schema, partition // columns etc. in table properties, so that we can work around the Hive metastore issue // about not case preserving and make Hive serde table support mixed-case column names. - properties = tableDef.properties ++ tableMetaToTableProps(tableDef)) + properties = tableDefinition.properties ++ tableMetaToTableProps(tableDefinition)) client.createTable(tableWithDataSourceProps, ignoreIfExists) } else { createDataSourceTable( - tableDef.withNewStorage(locationUri = tableLocation), + tableDefinition.withNewStorage(locationUri = tableLocation), ignoreIfExists) } } 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 6b6a2b49d163..842d72646f6c 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 @@ -49,8 +49,6 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi } override def afterEach(): Unit = { - sparkContext.conf.set( - SQLConf.PARQUET_TABLE_INCLUDE_TIMEZONE.key, "false") try { // drop all databases, tables and functions after each test spark.sessionState.catalog.reset() @@ -161,34 +159,30 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi // Check creating parquet tables with timestamps, writing data into them, and reading it back out // under a variety of conditions: - // * global conf for setting table tz by default // * tables with explicit tz and those without // * altering table properties directly // * variety of timezones, local & non-local - Seq(false, true).foreach { setTableTzByDefault => - testCreateWriteRead("no_tz", None, setTableTzByDefault) - val localTz = TimeZone.getDefault.getID() - testCreateWriteRead("local", Some(localTz), setTableTzByDefault) - // check with a variety of timezones. The unit tests currently are configured to always use - // America/Los_Angeles, but even if they didn't, we'd be sure to cover a non-local timezone. - Seq( - "UTC" -> "UTC", - "LA" -> "America/Los_Angeles", - "Berlin" -> "Europe/Berlin" - ).foreach { case (tableName, zone) => - if (zone != localTz) { - testCreateWriteRead(tableName, Some(zone), setTableTzByDefault) - } + testCreateWriteRead("no_tz", None) + val localTz = TimeZone.getDefault.getID() + testCreateWriteRead("local", Some(localTz)) + // check with a variety of timezones. The unit tests currently are configured to always use + // America/Los_Angeles, but even if they didn't, we'd be sure to cover a non-local timezone. + Seq( + "UTC" -> "UTC", + "LA" -> "America/Los_Angeles", + "Berlin" -> "Europe/Berlin" + ).foreach { case (tableName, zone) => + if (zone != localTz) { + testCreateWriteRead(tableName, Some(zone)) } } private def testCreateWriteRead( baseTable: String, - explicitTz: Option[String], - setTableTzByDefault: Boolean): Unit = { - testCreateAlterTablesWithTimezone(baseTable, explicitTz, setTableTzByDefault) - testWriteTablesWithTimezone(baseTable, explicitTz, setTableTzByDefault) - testReadTablesWithTimezone(baseTable, explicitTz, setTableTzByDefault) + explicitTz: Option[String]): Unit = { + testCreateAlterTablesWithTimezone(baseTable, explicitTz) + testWriteTablesWithTimezone(baseTable, explicitTz) + testReadTablesWithTimezone(baseTable, explicitTz) } private def checkHasTz(table: String, tz: Option[String]): Unit = { @@ -198,20 +192,18 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi private def testCreateAlterTablesWithTimezone( baseTable: String, - explicitTz: Option[String], - setTableTzByDefault: Boolean): Unit = { - test(s"SPARK-12297: Create and Alter Parquet tables and timezones; " + - s"setTzByDefault = $setTableTzByDefault; explicitTz = $explicitTz") { + explicitTz: Option[String]): Unit = { + + test(s"SPARK-12297: Create and Alter Parquet tables and timezones; explicitTz = $explicitTz") { // we're cheating a bit here, in general SparkConf isn't meant to be set at runtime, // but its OK in this case, and lets us run this test, because these tests don't like // creating multiple HiveContexts in the same jvm val key = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY - sparkContext.conf.set(SQLConf.PARQUET_TABLE_INCLUDE_TIMEZONE.key, - setTableTzByDefault.toString) withTable(baseTable, s"like_$baseTable", s"select_$baseTable") { val localTz = TimeZone.getDefault() val localTzId = localTz.getID() - val defaultTz = if (setTableTzByDefault) Some(localTzId) else None + // 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 => raw"""TBLPROPERTIES ($key="$tz")""" @@ -265,18 +257,15 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi private def testWriteTablesWithTimezone( baseTable: String, - explicitTz: Option[String], - setTableTzByDefault: Boolean): Unit = { + explicitTz: Option[String]) : Unit = { val key = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY - test(s"SPARK-12297: Write to Parquet tables with Timestamps; " + - s"setTzByDefault = $setTableTzByDefault; explicitTz = $explicitTz") { - sparkContext.conf.set( - SQLConf.PARQUET_TABLE_INCLUDE_TIMEZONE.key, setTableTzByDefault.toString) + test(s"SPARK-12297: Write to Parquet tables with Timestamps; explicitTz = $explicitTz") { withTable(s"saveAsTable_$baseTable", s"insert_$baseTable") { val localTz = TimeZone.getDefault() val localTzId = localTz.getID() - val defaultTz = if (setTableTzByDefault) Some(localTzId) else None + // If we ever add a property to set the table timezone by default, defaultTz would change + val defaultTz = None val expectedTableTz = explicitTz.orElse(defaultTz) // check that created tables have correct TBLPROPERTIES val tblProperties = explicitTz.map { @@ -333,13 +322,9 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi private def testReadTablesWithTimezone( baseTable: String, - explicitTz: Option[String], - setTableTzByDefault: Boolean): Unit = { + explicitTz: Option[String]): Unit = { val key = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY - test(s"SPARK-12297: Read from Parquet tables with Timestamps; " + - s"setTzByDefault = $setTableTzByDefault; explicitTz = $explicitTz") { - sparkContext.conf.set( - SQLConf.PARQUET_TABLE_INCLUDE_TIMEZONE.key, setTableTzByDefault.toString) + test(s"SPARK-12297: Read from Parquet tables with Timestamps; explicitTz = $explicitTz") { withTable(s"external_$baseTable") { // TODO check predicate pushdown // we intentionally save this data directly, without creating a table, so we can From a96806fa4011e28dcc875d0fb519501013b91a1b Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 24 Mar 2017 15:22:17 -0500 Subject: [PATCH 26/40] fixup --- .../spark/sql/catalyst/util/DateTimeUtils.scala | 5 +++++ .../apache/spark/sql/hive/HiveExternalCatalog.scala | 11 ++++++++++- 2 files changed, 15 insertions(+), 1 deletion(-) 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 9b94c1e2b40b..197ef77d1550 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 @@ -479,6 +479,11 @@ object DateTimeUtils { false } + lazy val validTimezones = TimeZone.getAvailableIDs().toSet + def isValidTimezone(timezoneId: String): Boolean = { + validTimezones.contains(timezoneId) + } + /** * Returns the microseconds since year zero (-17999) from microseconds since epoch. */ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index 33b21be37203..9add2b88c420 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -38,9 +38,10 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.escapePathName import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.ColumnStat -import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.PartitioningUtils +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.hive.client.HiveClient import org.apache.spark.sql.internal.HiveSerDe import org.apache.spark.sql.internal.StaticSQLConf._ @@ -206,6 +207,14 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat throw new TableAlreadyExistsException(db = db, table = table) } + val tableTz = tableDefinition.properties.get(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY) + tableTz.foreach { tz => + if (!DateTimeUtils.isValidTimezone(tz)) { + throw new AnalysisException(s"Cannot set" + + s" ${ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY} to invalid timezone $tz") + } + } + if (tableDefinition.tableType == VIEW) { client.createTable(tableDefinition, ignoreIfExists) } else { From 7582b2cae3f816ba9ebd75f6a1bc5ac3157f3070 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 4 Apr 2017 16:01:12 -0500 Subject: [PATCH 27/40] predicate pushdown tests --- .../hive/ParquetHiveCompatibilitySuite.scala | 75 +++++++++++++++---- 1 file changed, 61 insertions(+), 14 deletions(-) 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 842d72646f6c..cdb9bcc38289 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 @@ -20,6 +20,8 @@ package org.apache.spark.sql.hive import java.sql.Timestamp import java.util.TimeZone +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.parquet.hadoop.ParquetFileReader import org.scalatest.BeforeAndAfterEach import org.apache.spark.sql.{AnalysisException, Dataset, Row} @@ -246,8 +248,8 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi private def createRawData(): Dataset[Timestamp] = { import spark.implicits._ val originalTsStrings = Seq( - "2015-12-31 23:50:59.123", "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" ) @@ -326,13 +328,25 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi val key = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY test(s"SPARK-12297: Read from Parquet tables with Timestamps; explicitTz = $explicitTz") { withTable(s"external_$baseTable") { - // TODO check predicate pushdown // we intentionally save this data directly, without creating a table, so we can // see that the data is read back differently depending on table properties val localTz = TimeZone.getDefault() val rawData = createRawData() + // adjust the raw parquet data based on the timezones, so that it should get read back the + // same way + val adjustedRawData = explicitTz match { + case Some(tzId) => + val storageTz = TimeZone.getTimeZone(tzId) + import spark.implicits._ + rawData.map { ts => + val t = ts.getTime() + new Timestamp(t + storageTz.getOffset(t) - localTz.getOffset(t)) + } + case _ => + rawData + } withTempPath { path => - rawData.write.parquet(path.getCanonicalPath) + adjustedRawData.write.parquet(path.getCanonicalPath) val options = Map("path" -> path.getCanonicalPath) ++ explicitTz.map { tz => Map(key -> tz) }.getOrElse(Map()) @@ -348,19 +362,52 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi val collectedFromExternal = spark.sql(s"select value from external_$baseTable").collect() .map(_.getAs[Timestamp](0)) - val expTimestamps = explicitTz match { - case Some(tzId) => - val storageTz = TimeZone.getTimeZone(tzId) - rawData.collect().map { ts => - val t = ts.getTime() - new Timestamp(t - storageTz.getOffset(t) + localTz.getOffset(t)) - } - case _ => - // no modification to raw data in parquet - rawData.collect() - } + val expTimestamps = rawData.collect() assert(collectedFromExternal === expTimestamps, s"collected = ${collectedFromExternal.mkString(",")}") + + // Now test that the behavior is still correct even with a filter which could get + // pushed down into parquet. You would expect we'd need to do extra handling of + // this case, but it turns out we don't, because parquet does not read statistics + // from int96 fields, as they are unsigned. See + // scalastyle:off line.size.limit + // https://github.com/apache/parquet-mr/blob/2fd62ee4d524c270764e9b91dca72e5cf1a005b7/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java#L419 + // https://github.com/apache/parquet-mr/blob/2fd62ee4d524c270764e9b91dca72e5cf1a005b7/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java#L348 + // scalastyle:on line.size.limit + // + // Just to be defensive in case anything ever changes in parquet, this test checks + // the assumption on column stats, and also the end-to-end behavior. + + val hadoopConf = sparkContext.hadoopConfiguration + val fs = FileSystem.get(hadoopConf) + val parts = fs.listStatus(new Path(path.getCanonicalPath)) + .filter(_.getPath().getName().endsWith(".parquet")) + assert(parts.size == 1) + val oneFooter = ParquetFileReader.readFooter(hadoopConf, parts.head.getPath) + assert(oneFooter.getFileMetaData.getSchema.getColumns.size == 1) + val oneBlockMeta = oneFooter.getBlocks().get(0) + val oneBlockColumnMeta = oneBlockMeta.getColumns().get(0) + val columnStats = oneBlockColumnMeta.getStatistics + // Column stats are written, but they are ignored when the data is read back as + // mentioned above, b/c int96 is unsigned. This assert makes sure this holds even + // if we change parquet versions (if eg. there were ever statistics even on unsigned + // columns). + assert(columnStats.isEmpty) + + // 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 value from external_$baseTable where value $comparison '$value'" + val countWithFilter = spark + .sql(query) + .count() + assert(countWithFilter === 4, query) + } + } } } From 5817064ab39a3cc454faf31ddf00730d4ddb6c09 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 6 Apr 2017 13:23:11 -0500 Subject: [PATCH 28/40] minor cleanup --- .../spark/sql/hive/ParquetHiveCompatibilitySuite.scala | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) 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 cdb9bcc38289..d6f76acc7540 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 @@ -367,9 +367,9 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi s"collected = ${collectedFromExternal.mkString(",")}") // Now test that the behavior is still correct even with a filter which could get - // pushed down into parquet. You would expect we'd need to do extra handling of - // this case, but it turns out we don't, because parquet does not read statistics - // from int96 fields, as they are unsigned. See + // pushed down into parquet. We don't need extra handling for pushed down + // predicates because (a) in ParquetFilters, we ignore TimestampType and (b) parquet + // does not read statistics from int96 fields, as they are unsigned. See // scalastyle:off line.size.limit // https://github.com/apache/parquet-mr/blob/2fd62ee4d524c270764e9b91dca72e5cf1a005b7/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java#L419 // https://github.com/apache/parquet-mr/blob/2fd62ee4d524c270764e9b91dca72e5cf1a005b7/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java#L348 @@ -402,9 +402,7 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi ).foreach { case (comparison, value) => val query = s"select value from external_$baseTable where value $comparison '$value'" - val countWithFilter = spark - .sql(query) - .count() + val countWithFilter = spark.sql(query).count() assert(countWithFilter === 4, query) } From be134befac5c12832dc0348c1d4a4cc2bc3f3d39 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Sun, 9 Apr 2017 22:51:57 -0500 Subject: [PATCH 29/40] fix merge --- .../scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 1da27eb8c264..301006c66a31 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.{QualifiedTableName, TableIdentifier} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode._ import org.apache.spark.sql.types._ From d15b660bbb62101f32b8040a37a44534ef6fdd12 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 10 Apr 2017 09:37:28 -0500 Subject: [PATCH 30/40] swap conversion logic --- .../datasources/parquet/VectorizedColumnReader.java | 4 ++-- .../datasources/parquet/ParquetRowConverter.scala | 10 +++++----- .../datasources/parquet/ParquetWriteSupport.scala | 2 +- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java index eb6ec5337adc..6124e0ff1cc8 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java @@ -303,7 +303,7 @@ private void decodeDictionaryIds(int rowId, int num, ColumnVector column, // TODO: Convert dictionary of Binaries to dictionary of Longs if (!column.isNullAt(i)) { Binary v = dictionary.decodeToBinary(dictionaryIds.getDictId(i)); - column.putLong(i, ParquetRowConverter.binaryToSQLTimestamp(v, storageTz, localTz)); + column.putLong(i, ParquetRowConverter.binaryToSQLTimestamp(v, localTz, storageTz)); } } } else { @@ -436,7 +436,7 @@ private void readBinaryBatch(int rowId, int num, ColumnVector column) throws IOE if (defColumn.readInteger() == maxDefLevel) { column.putLong(rowId + i, // Read 12 bytes for INT96 - ParquetRowConverter.binaryToSQLTimestamp(data.readBinary(12), storageTz, localTz)); + ParquetRowConverter.binaryToSQLTimestamp(data.readBinary(12), localTz, storageTz)); } else { column.putNull(rowId + i); } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala index 8869f1d9d864..599885d35ff1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala @@ -276,8 +276,8 @@ private[parquet] class ParquetRowConverter( new ParquetPrimitiveConverter(updater) { // Converts nanosecond timestamps stored as INT96 override def addBinary(value: Binary): Unit = { - val timestamp = ParquetRowConverter.binaryToSQLTimestamp(value, storageTz = storageTz, - localTz = localTz) + val timestamp = ParquetRowConverter.binaryToSQLTimestamp(value, fromTz = localTz, + toTz = storageTz) updater.setLong(timestamp) } } @@ -688,7 +688,7 @@ private[parquet] object ParquetRowConverter { * @param binary * @return */ - def binaryToSQLTimestamp(binary: Binary, storageTz: TimeZone, localTz: TimeZone): SQLTimestamp = { + def binaryToSQLTimestamp(binary: Binary, fromTz: TimeZone, toTz: TimeZone): SQLTimestamp = { assert(binary.length() == 12, s"Timestamps (with nanoseconds) are expected to be stored in" + s" 12-byte long binaries. Found a ${binary.length()}-byte binary instead.") val buffer = binary.toByteBuffer.order(ByteOrder.LITTLE_ENDIAN) @@ -696,8 +696,8 @@ private[parquet] object ParquetRowConverter { val julianDay = buffer.getInt val utcEpochMicros = DateTimeUtils.fromJulianDay(julianDay, timeOfDayNanos) // avoid expensive time logic if possible. - if (storageTz.getID() != localTz.getID()) { - DateTimeUtils.convertTz(utcEpochMicros, storageTz, localTz) + if (fromTz.getID() != toTz.getID()) { + DateTimeUtils.convertTz(utcEpochMicros, fromTz, toTz) } else { utcEpochMicros } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala index 63998f3fb931..ca01522579b6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala @@ -193,7 +193,7 @@ private[parquet] class ParquetWriteSupport extends WriteSupport[InternalRow] wit val adjustedMicros = if (localTz.getID() == storageTz.getID()) { rawMicros } else { - DateTimeUtils.convertTz(rawMicros, localTz, storageTz) + DateTimeUtils.convertTz(rawMicros, storageTz, localTz) } val (julianDay, timeOfDayNanos) = DateTimeUtils.toJulianDay(adjustedMicros) val buf = ByteBuffer.wrap(timestampBuffer) From 283b1c7183e2b32ea62b1c47dc9cab85ae000c33 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 10 Apr 2017 22:26:24 -0500 Subject: [PATCH 31/40] update tests a bunch; tests session timezones, and tests are easier to reason about now --- .../hive/ParquetHiveCompatibilitySuite.scala | 228 +++++++++++------- 1 file changed, 144 insertions(+), 84 deletions(-) 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 d6f76acc7540..2b612915efb1 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 @@ -22,14 +22,16 @@ 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.scalatest.BeforeAndAfterEach -import org.apache.spark.sql.{AnalysisException, Dataset, Row} +import org.apache.spark.sql.{AnalysisException, Dataset, Row, SparkSession} 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.{StructType, TimestampType} +import org.apache.spark.sql.types.{StringType, StructField, StructType, TimestampType} class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHiveSingleton with BeforeAndAfterEach { @@ -159,32 +161,44 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi "ARRAY>") } + val testTimezones = Seq( + "UTC" -> "UTC", + "LA" -> "America/Los_Angeles", + "Berlin" -> "Europe/Berlin" + ) // 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 - testCreateWriteRead("no_tz", None) - val localTz = TimeZone.getDefault.getID() - testCreateWriteRead("local", Some(localTz)) - // check with a variety of timezones. The unit tests currently are configured to always use - // America/Los_Angeles, but even if they didn't, we'd be sure to cover a non-local timezone. - Seq( - "UTC" -> "UTC", - "LA" -> "America/Los_Angeles", - "Berlin" -> "Europe/Berlin" - ).foreach { case (tableName, zone) => - if (zone != localTz) { - testCreateWriteRead(tableName, Some(zone)) + val sessionTimezones = testTimezones.map(_._2).map(Some(_)) ++ Seq(None) + sessionTimezones.foreach { sessionTzOpt => + val sparkSession = spark.newSession() + sessionTzOpt.foreach { tz => sparkSession.conf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, tz) } + testCreateWriteRead(sparkSession, "no_tz", None, sessionTzOpt) + val localTz = TimeZone.getDefault.getID() + testCreateWriteRead(sparkSession, "local", Some(localTz), sessionTzOpt) + // check with a variety of timezones. The unit tests currently are configured to always use + // America/Los_Angeles, but even if they didn't, we'd be sure to cover a non-local timezone. + Seq( + "UTC" -> "UTC", + "LA" -> "America/Los_Angeles", + "Berlin" -> "Europe/Berlin" + ).foreach { case (tableName, zone) => + if (zone != localTz) { + testCreateWriteRead(sparkSession, tableName, Some(zone), sessionTzOpt) + } } } private def testCreateWriteRead( + sparkSession: SparkSession, baseTable: String, - explicitTz: Option[String]): Unit = { - testCreateAlterTablesWithTimezone(baseTable, explicitTz) - testWriteTablesWithTimezone(baseTable, explicitTz) - testReadTablesWithTimezone(baseTable, explicitTz) + explicitTz: Option[String], + sessionTzOpt: Option[String]): Unit = { + testCreateAlterTablesWithTimezone(sparkSession, baseTable, explicitTz, sessionTzOpt) + testWriteTablesWithTimezone(sparkSession, baseTable, explicitTz, sessionTzOpt) + testReadTablesWithTimezone(sparkSession, baseTable, explicitTz, sessionTzOpt) } private def checkHasTz(table: String, tz: Option[String]): Unit = { @@ -193,13 +207,12 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi } private def testCreateAlterTablesWithTimezone( + spark: SparkSession, baseTable: String, - explicitTz: Option[String]): Unit = { - - test(s"SPARK-12297: Create and Alter Parquet tables and timezones; explicitTz = $explicitTz") { - // we're cheating a bit here, in general SparkConf isn't meant to be set at runtime, - // but its OK in this case, and lets us run this test, because these tests don't like - // creating multiple HiveContexts in the same jvm + explicitTz: Option[String], + sessionTzOpt: Option[String]): Unit = { + test(s"SPARK-12297: Create and Alter Parquet tables and timezones; explicitTz = $explicitTz; " + + s"sessionTzOpt = $sessionTzOpt") { val key = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY withTable(baseTable, s"like_$baseTable", s"select_$baseTable") { val localTz = TimeZone.getDefault() @@ -245,55 +258,96 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi } } - private def createRawData(): Dataset[Timestamp] = { - import spark.implicits._ + val desiredTimestampStrings = Seq( + "2015-12-31 23:50:59.123", + "2015-12-31 22:49: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 originalTz = TimeZone.getDefault + val timestampTimezoneToMillis = try { + (for { + timestampString <- desiredTimestampStrings + timezone <- Seq("America/Los_Angeles", "Europe/Berlin", "UTC").map { + TimeZone.getTimeZone(_) + } + } yield { + TimeZone.setDefault(timezone) + val timestamp = Timestamp.valueOf(timestampString) + (timestampString, timezone.getID()) -> timestamp.getTime() + }).toMap + } finally { + TimeZone.setDefault(originalTz) + } + + private def createRawData(spark: SparkSession): Dataset[(String, Timestamp)] = { val originalTsStrings = 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" ) - spark.createDataset( - originalTsStrings.map { x => java.sql.Timestamp.valueOf(x) }) + val rowRdd = spark.sparkContext.parallelize(originalTsStrings, 1).map(Row(_)) + val schema = StructType(Seq( + StructField("display", StringType, true) + )) + val df = spark.createDataFrame(rowRdd, schema) + // this will get the millis corresponding to the display time given the current *session* + // timezone. + import spark.implicits._ + val r = df.withColumn("ts", expr("cast(display as timestamp)")).map { row => + (row.getAs[String](0), row.getAs[Timestamp](1)) + } + val sessionTzA = spark.sparkContext.getConf.get(SQLConf.SESSION_LOCAL_TIMEZONE) + val sessionTzB = spark.conf.get(SQLConf.SESSION_LOCAL_TIMEZONE) + val row = r.collect().find{_._1 == "2015-12-31 22:49:59.123"}.head + logWarning(s"with session tz = ${(sessionTzA, sessionTzB)}, " + + s"'2015-12-31 22:49:59.123' --> ${row._2.getTime()}") + r } private def testWriteTablesWithTimezone( + spark: SparkSession, baseTable: String, - explicitTz: Option[String]) : Unit = { + explicitTz: Option[String], + sessionTzOpt: Option[String]) : Unit = { val key = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY - test(s"SPARK-12297: Write to Parquet tables with Timestamps; explicitTz = $explicitTz") { + test(s"SPARK-12297: Write to Parquet tables with Timestamps; explicitTz = $explicitTz; " + + s"sessionTzOpt = $sessionTzOpt") { withTable(s"saveAsTable_$baseTable", s"insert_$baseTable") { - val localTz = TimeZone.getDefault() - val localTzId = localTz.getID() - // If we ever add a property to set the table timezone by default, defaultTz would change - val defaultTz = None - val expectedTableTz = explicitTz.orElse(defaultTz) + val sessionTzId = sessionTzOpt.getOrElse(TimeZone.getDefault().getID()) // check that created tables have correct TBLPROPERTIES val tblProperties = explicitTz.map { tz => raw"""TBLPROPERTIES ($key="$tz")""" }.getOrElse("") - val rawData = createRawData() + val rawData = createRawData(spark) // Check writing data out. // We write data into our tables, and then check the raw parquet files to see whether // the correct conversion was applied. rawData.write.saveAsTable(s"saveAsTable_$baseTable") - checkHasTz(s"saveAsTable_$baseTable", defaultTz) + checkHasTz(s"saveAsTable_$baseTable", None) spark.sql( raw"""CREATE TABLE insert_$baseTable ( + | display string, | ts timestamp | ) | STORED AS PARQUET | $tblProperties """.stripMargin) - checkHasTz(s"insert_$baseTable", expectedTableTz) + checkHasTz(s"insert_$baseTable", explicitTz) rawData.write.insertInto(s"insert_$baseTable") - val readFromTable = spark.table(s"insert_$baseTable").collect() - .map(_.getAs[Timestamp](0)) // no matter what, roundtripping via the table should leave the data unchanged - assert(readFromTable === rawData.collect()) + val readFromTable = spark.table(s"insert_$baseTable").collect() + .map { row => (row.getAs[String](0), row.getAs[Timestamp](1)).toString() }.sorted + assert(readFromTable === rawData.collect().map(_.toString()).sorted) // Now we load the raw parquet data on disk, and check if it was adjusted correctly. // Note that we only store the timezone in the table property, so when we read the @@ -302,49 +356,45 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi val onDiskLocation = spark.sessionState.catalog .getTableMetadata(TableIdentifier(s"insert_$baseTable")).location.getPath val readFromDisk = spark.read.parquet(onDiskLocation).collect() - .map(_.getAs[Timestamp](0)) - val expectedReadFromDisk = expectedTableTz match { - case Some(tzId) => - // We should have shifted the data from our local timezone to the storage timezone - // when we saved the data. - val storageTz = TimeZone.getTimeZone(tzId) - rawData.collect().map { ts => - val t = ts.getTime() - new Timestamp(t + storageTz.getOffset(t) - localTz.getOffset(t)) - } - case _ => - rawData.collect() + val storageTzId = explicitTz.getOrElse(sessionTzId) + readFromDisk.foreach { row => + val displayTime = row.getAs[String](0) + val millis = row.getAs[Timestamp](1).getTime() + val expectedMillis = timestampTimezoneToMillis((displayTime, storageTzId)) + assert(expectedMillis === millis, s"Display time '$displayTime' was stored incorrectly " + + s"with sessionTz = ${sessionTzOpt}; Got $millis, expected $expectedMillis " + + s"(delta = ${millis - expectedMillis})") } - assert(readFromDisk === expectedReadFromDisk, - s"timestamps changed string format after reading back from parquet with " + - s"local = $localTzId & storage = $expectedTableTz") } } } private def testReadTablesWithTimezone( + spark: SparkSession, baseTable: String, - explicitTz: Option[String]): Unit = { + explicitTz: Option[String], + sessionTzOpt: Option[String]): Unit = { val key = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY - test(s"SPARK-12297: Read from Parquet tables with Timestamps; explicitTz = $explicitTz") { + test(s"SPARK-12297: Read from Parquet tables with Timestamps; explicitTz = $explicitTz; " + + s"sessionTzOpt = $sessionTzOpt") { withTable(s"external_$baseTable") { // we intentionally save this data directly, without creating a table, so we can - // see that the data is read back differently depending on table properties - val localTz = TimeZone.getDefault() - val rawData = createRawData() - // adjust the raw parquet data based on the timezones, so that it should get read back the - // same way - val adjustedRawData = explicitTz match { + // see that the data is read back differently depending on table properties. + // we'll save with adjusted millis, so that it should be the correct millis after reading + // back. + val rawData = createRawData(spark) + // to avoid closing over entire class + val timestampTimezoneToMillis = this.timestampTimezoneToMillis + import spark.implicits._ + val adjustedRawData = (explicitTz match { case Some(tzId) => - val storageTz = TimeZone.getTimeZone(tzId) - import spark.implicits._ - rawData.map { ts => - val t = ts.getTime() - new Timestamp(t + storageTz.getOffset(t) - localTz.getOffset(t)) + rawData.map { case (displayTime, _) => + val storageMillis = timestampTimezoneToMillis((displayTime, tzId)) + (displayTime, new Timestamp(storageMillis)) } case _ => rawData - } + }).withColumnRenamed("_1", "display").withColumnRenamed("_2", "ts") withTempPath { path => adjustedRawData.write.parquet(path.getCanonicalPath) val options = Map("path" -> path.getCanonicalPath) ++ @@ -353,18 +403,25 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi spark.catalog.createTable( tableName = s"external_$baseTable", source = "parquet", - schema = new StructType().add("value", TimestampType), + schema = new StructType().add("display", StringType).add("ts", TimestampType), options = options ) Seq(false, true).foreach { vectorized => withSQLConf((SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, vectorized.toString)) { withClue(s"vectorized = $vectorized;") { + val sessionTz = sessionTzOpt.getOrElse(TimeZone.getDefault().getID()) val collectedFromExternal = - spark.sql(s"select value from external_$baseTable").collect() - .map(_.getAs[Timestamp](0)) - val expTimestamps = rawData.collect() - assert(collectedFromExternal === expTimestamps, - s"collected = ${collectedFromExternal.mkString(",")}") + spark.sql(s"select display, ts from external_$baseTable").collect() + collectedFromExternal.foreach { row => + val displayTime = row.getAs[String](0) + val millis = row.getAs[Timestamp](1).getTime() + val expectedMillis = timestampTimezoneToMillis((displayTime, sessionTz)) + val delta = millis - expectedMillis + val deltaHours = delta / (1000L * 60 * 60) + assert(millis === expectedMillis, s"Display time '$displayTime' did not have " + + s"correct millis: was $millis, expected $expectedMillis; delta = $delta " + + s"($deltaHours hours)") + } // Now test that the behavior is still correct even with a filter which could get // pushed down into parquet. We don't need extra handling for pushed down @@ -382,16 +439,20 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi val fs = FileSystem.get(hadoopConf) val parts = fs.listStatus(new Path(path.getCanonicalPath)) .filter(_.getPath().getName().endsWith(".parquet")) + // grab the meta data from the parquet file. The next section of asserts just make + // sure the test is configured correctly. assert(parts.size == 1) val oneFooter = ParquetFileReader.readFooter(hadoopConf, parts.head.getPath) - assert(oneFooter.getFileMetaData.getSchema.getColumns.size == 1) + assert(oneFooter.getFileMetaData.getSchema.getColumns.size === 2) + assert(oneFooter.getFileMetaData.getSchema.getColumns.get(1).getType() === + PrimitiveTypeName.INT96) val oneBlockMeta = oneFooter.getBlocks().get(0) - val oneBlockColumnMeta = oneBlockMeta.getColumns().get(0) + val oneBlockColumnMeta = oneBlockMeta.getColumns().get(1) val columnStats = oneBlockColumnMeta.getStatistics - // Column stats are written, but they are ignored when the data is read back as - // mentioned above, b/c int96 is unsigned. This assert makes sure this holds even - // if we change parquet versions (if eg. there were ever statistics even on unsigned - // columns). + // This is the important assert. Column stats are written, but they are ignored + // when the data is read back as mentioned above, b/c int96 is unsigned. This + // assert makes sure this holds even if we change parquet versions (if eg. there + // were ever statistics even on unsigned columns). assert(columnStats.isEmpty) // These queries should return the entire dataset, but if the predicates were @@ -401,11 +462,10 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi "<" -> "2016-01-01 02:00:00" ).foreach { case (comparison, value) => val query = - s"select value from external_$baseTable where value $comparison '$value'" + s"select ts from external_$baseTable where ts $comparison '$value'" val countWithFilter = spark.sql(query).count() assert(countWithFilter === 4, query) } - } } } From 6ccaa924ae4fa3f299f5c98421f063b13ccc0a22 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 10 Apr 2017 22:26:55 -0500 Subject: [PATCH 32/40] session timezones --- .../parquet/VectorizedColumnReader.java | 22 +++++++++++++------ .../parquet/ParquetRowConverter.scala | 13 ++++++----- .../parquet/ParquetWriteSupport.scala | 20 +++++++++++------ .../spark/sql/hive/HiveMetastoreCatalog.scala | 12 +++++++--- 4 files changed, 44 insertions(+), 23 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java index 6124e0ff1cc8..305cfa7b32d2 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java @@ -32,6 +32,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils; import org.apache.spark.sql.execution.vectorized.ColumnVector; +import org.apache.spark.sql.internal.SQLConf; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.DecimalType; @@ -93,7 +94,7 @@ public class VectorizedColumnReader { private final PageReader pageReader; private final ColumnDescriptor descriptor; private final TimeZone storageTz; - private final TimeZone localTz = TimeZone.getDefault(); + private final TimeZone sessionTz; public VectorizedColumnReader(ColumnDescriptor descriptor, PageReader pageReader, Configuration conf) @@ -102,12 +103,19 @@ public VectorizedColumnReader(ColumnDescriptor descriptor, PageReader pageReader this.pageReader = pageReader; // If the table has a timezone property, apply the correct conversions. See SPARK-12297. // The conf is sometimes null in tests. - String tzString = + String sessionTzString = + conf == null ? null : conf.get(SQLConf.SESSION_LOCAL_TIMEZONE().key()); + if (sessionTzString == null || sessionTzString.isEmpty()) { + sessionTz = TimeZone.getDefault(); + } else { + sessionTz = TimeZone.getTimeZone(sessionTzString); + } + String storageTzString = conf == null ? null : conf.get(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY()); - if (tzString == null || tzString.isEmpty()) { - storageTz = localTz; + if (storageTzString == null || storageTzString.isEmpty()) { + storageTz = sessionTz; } else { - storageTz = TimeZone.getTimeZone(tzString); + storageTz = TimeZone.getTimeZone(storageTzString); } this.maxDefLevel = descriptor.getMaxDefinitionLevel(); @@ -303,7 +311,7 @@ private void decodeDictionaryIds(int rowId, int num, ColumnVector column, // TODO: Convert dictionary of Binaries to dictionary of Longs if (!column.isNullAt(i)) { Binary v = dictionary.decodeToBinary(dictionaryIds.getDictId(i)); - column.putLong(i, ParquetRowConverter.binaryToSQLTimestamp(v, localTz, storageTz)); + column.putLong(i, ParquetRowConverter.binaryToSQLTimestamp(v, sessionTz, storageTz)); } } } else { @@ -436,7 +444,7 @@ private void readBinaryBatch(int rowId, int num, ColumnVector column) throws IOE if (defColumn.readInteger() == maxDefLevel) { column.putLong(rowId + i, // Read 12 bytes for INT96 - ParquetRowConverter.binaryToSQLTimestamp(data.readBinary(12), localTz, storageTz)); + ParquetRowConverter.binaryToSQLTimestamp(data.readBinary(12), sessionTz, storageTz)); } else { column.putNull(rowId + i); } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala index 599885d35ff1..49fb18732a30 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala @@ -36,6 +36,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, DateTimeUtils, GenericArrayData} import org.apache.spark.sql.catalyst.util.DateTimeUtils.SQLTimestamp +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -266,17 +267,17 @@ private[parquet] class ParquetRowConverter( case TimestampType => // TODO Implements `TIMESTAMP_MICROS` once parquet-mr has that. // If the table has a timezone property, apply the correct conversions. See SPARK-12297. - val tzString = hadoopConf.get(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY) - val localTz = TimeZone.getDefault() - val storageTz = if (tzString == null) { - localTz + val sessionTz = TimeZone.getTimeZone(hadoopConf.get(SQLConf.SESSION_LOCAL_TIMEZONE.key)) + val storageTzString = hadoopConf.get(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY) + val storageTz = if (storageTzString == null) { + sessionTz } else { - TimeZone.getTimeZone(tzString) + TimeZone.getTimeZone(storageTzString) } new ParquetPrimitiveConverter(updater) { // Converts nanosecond timestamps stored as INT96 override def addBinary(value: Binary): Unit = { - val timestamp = ParquetRowConverter.binaryToSQLTimestamp(value, fromTz = localTz, + val timestamp = ParquetRowConverter.binaryToSQLTimestamp(value, fromTz = sessionTz, toTz = storageTz) updater.setLong(timestamp) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala index ca01522579b6..679ed8e361b7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala @@ -77,7 +77,7 @@ private[parquet] class ParquetWriteSupport extends WriteSupport[InternalRow] wit private val decimalBuffer = new Array[Byte](minBytesForPrecision(DecimalType.MAX_PRECISION)) private var storageTz: TimeZone = _ - private var localTz: TimeZone = TimeZone.getDefault() + private var sessionTz: TimeZone = _ override def init(configuration: Configuration): WriteContext = { val schemaString = configuration.get(ParquetWriteSupport.SPARK_ROW_SCHEMA) @@ -96,11 +96,17 @@ private[parquet] class ParquetWriteSupport extends WriteSupport[InternalRow] wit this.rootFieldWriters = schema.map(_.dataType).map(makeWriter) // If the table has a timezone property, apply the correct conversions. See SPARK-12297. - val tzString = configuration.get(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY) - storageTz = if (tzString == null || tzString == "") { - localTz + val sessionTzString = configuration.get(SQLConf.SESSION_LOCAL_TIMEZONE.key) + sessionTz = if (sessionTzString == null || sessionTzString == "") { + TimeZone.getDefault() } else { - TimeZone.getTimeZone(tzString) + TimeZone.getTimeZone(sessionTzString) + } + val storageTzString = configuration.get(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY) + storageTz = if (storageTzString == null || storageTzString == "") { + sessionTz + } else { + TimeZone.getTimeZone(storageTzString) } val messageType = new ParquetSchemaConverter(configuration).convert(schema) @@ -190,10 +196,10 @@ private[parquet] class ParquetWriteSupport extends WriteSupport[InternalRow] wit // NOTE: Starting from Spark 1.5, Spark SQL `TimestampType` only has microsecond // precision. Nanosecond parts of timestamp values read from INT96 are simply stripped. val rawMicros = row.getLong(ordinal) - val adjustedMicros = if (localTz.getID() == storageTz.getID()) { + val adjustedMicros = if (sessionTz.getID() == storageTz.getID()) { rawMicros } else { - DateTimeUtils.convertTz(rawMicros, storageTz, localTz) + DateTimeUtils.convertTz(rawMicros, storageTz, sessionTz) } val (julianDay, timeOfDayNanos) = DateTimeUtils.toJulianDay(adjustedMicros) val buf = ByteBuffer.wrap(timestampBuffer) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 301006c66a31..9562d10bb003 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -30,6 +30,7 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode._ import org.apache.spark.sql.types._ @@ -195,8 +196,13 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log val logicalRelation = cached.getOrElse { // We add the timezone to the relation options, which automatically gets injected into // the hadoopConf for the Parquet Converters - val tzKey = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY - val tz = relation.tableMeta.properties.getOrElse(tzKey, "") + val storageTzKey = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY + val storageTz = relation.tableMeta.properties.getOrElse(storageTzKey, "") + val sessionTz = sparkSession.sessionState.conf.sessionLocalTimeZone + val extraTzOptions = Map( + storageTzKey -> storageTz, + SQLConf.SESSION_LOCAL_TIMEZONE.key -> sessionTz + ) val (dataSchema, updatedTable) = inferIfNeeded(relation, options, fileFormat) val created = LogicalRelation( @@ -206,7 +212,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log userSpecifiedSchema = Option(dataSchema), // We don't support hive bucketed tables, only ones we write out. bucketSpec = None, - options = options ++ Map(tzKey -> tz), + options = options ++ extraTzOptions, className = fileType).resolveRelation(), table = updatedTable) From 71c7e60b62269ab4290e4c09263585c92e6e40bb Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 10 Apr 2017 22:30:01 -0500 Subject: [PATCH 33/40] cleanup --- .../spark/sql/hive/ParquetHiveCompatibilitySuite.scala | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) 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 2b612915efb1..80f7bf3db894 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 @@ -300,15 +300,9 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi // this will get the millis corresponding to the display time given the current *session* // timezone. import spark.implicits._ - val r = df.withColumn("ts", expr("cast(display as timestamp)")).map { row => + df.withColumn("ts", expr("cast(display as timestamp)")).map { row => (row.getAs[String](0), row.getAs[Timestamp](1)) } - val sessionTzA = spark.sparkContext.getConf.get(SQLConf.SESSION_LOCAL_TIMEZONE) - val sessionTzB = spark.conf.get(SQLConf.SESSION_LOCAL_TIMEZONE) - val row = r.collect().find{_._1 == "2015-12-31 22:49:59.123"}.head - logWarning(s"with session tz = ${(sessionTzA, sessionTzB)}, " + - s"'2015-12-31 22:49:59.123' --> ${row._2.getTime()}") - r } private def testWriteTablesWithTimezone( From e4e88a5c4f7d900a87beeb1df33efd33f260cfa3 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 19 Apr 2017 15:47:22 -0500 Subject: [PATCH 34/40] partial review feedback --- .../parquet/VectorizedColumnReader.java | 2 +- .../parquet/ParquetRowConverter.scala | 10 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 24 +-- .../hive/ParquetHiveCompatibilitySuite.scala | 159 +++++++++--------- 4 files changed, 98 insertions(+), 97 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java index 305cfa7b32d2..dabbc2b6387e 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java @@ -106,7 +106,7 @@ public VectorizedColumnReader(ColumnDescriptor descriptor, PageReader pageReader String sessionTzString = conf == null ? null : conf.get(SQLConf.SESSION_LOCAL_TIMEZONE().key()); if (sessionTzString == null || sessionTzString.isEmpty()) { - sessionTz = TimeZone.getDefault(); + sessionTz = DateTimeUtils.defaultTimeZone(); } else { sessionTz = TimeZone.getTimeZone(sessionTzString); } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala index 49fb18732a30..3cdd1b882886 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala @@ -267,13 +267,11 @@ private[parquet] class ParquetRowConverter( case TimestampType => // TODO Implements `TIMESTAMP_MICROS` once parquet-mr has that. // If the table has a timezone property, apply the correct conversions. See SPARK-12297. - val sessionTz = TimeZone.getTimeZone(hadoopConf.get(SQLConf.SESSION_LOCAL_TIMEZONE.key)) + val sessionTsString = hadoopConf.get(SQLConf.SESSION_LOCAL_TIMEZONE.key) + val sessionTz = Option(sessionTsString).map(TimeZone.getTimeZone(_)) + .getOrElse(DateTimeUtils.defaultTimeZone()) val storageTzString = hadoopConf.get(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY) - val storageTz = if (storageTzString == null) { - sessionTz - } else { - TimeZone.getTimeZone(storageTzString) - } + val storageTz = Option(storageTzString).map(TimeZone.getTimeZone(_)).getOrElse(sessionTz) new ParquetPrimitiveConverter(updater) { // Converts nanosecond timestamps stored as INT96 override def addBinary(value: Binary): Unit = { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 9562d10bb003..490bce11c6a8 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -176,7 +176,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log // We don't support hive bucketed tables, only ones we write out. bucketSpec = None, fileFormat = fileFormat, - options = options)(sparkSession = sparkSession) + options = options ++ getStorageTzOptions(relation))(sparkSession = sparkSession) val created = LogicalRelation(fsRelation, updatedTable) tableRelationCache.put(tableIdentifier, created) created @@ -194,15 +194,6 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log fileFormatClass, None) val logicalRelation = cached.getOrElse { - // We add the timezone to the relation options, which automatically gets injected into - // the hadoopConf for the Parquet Converters - val storageTzKey = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY - val storageTz = relation.tableMeta.properties.getOrElse(storageTzKey, "") - val sessionTz = sparkSession.sessionState.conf.sessionLocalTimeZone - val extraTzOptions = Map( - storageTzKey -> storageTz, - SQLConf.SESSION_LOCAL_TIMEZONE.key -> sessionTz - ) val (dataSchema, updatedTable) = inferIfNeeded(relation, options, fileFormat) val created = LogicalRelation( @@ -212,7 +203,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log userSpecifiedSchema = Option(dataSchema), // We don't support hive bucketed tables, only ones we write out. bucketSpec = None, - options = options ++ extraTzOptions, + options = options ++ getStorageTzOptions(relation), className = fileType).resolveRelation(), table = updatedTable) @@ -233,6 +224,17 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log result.copy(output = newOutput) } + private def getStorageTzOptions(relation: CatalogRelation): Map[String, String] = { + // We add the table timezone to the relation options, which automatically gets injected into the + // hadoopConf for the Parquet Converters + val storageTzKey = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY + val storageTz = relation.tableMeta.properties.getOrElse(storageTzKey, "") + val sessionTz = sparkSession.sessionState.conf.sessionLocalTimeZone + Map( + storageTzKey -> storageTz + ) + } + private def inferIfNeeded( relation: CatalogRelation, options: Map[String, String], 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 80f7bf3db894..1d0309db42ca 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 @@ -259,8 +259,8 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi } val desiredTimestampStrings = Seq( - "2015-12-31 23:50:59.123", "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" ) @@ -286,13 +286,7 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi } private def createRawData(spark: SparkSession): Dataset[(String, Timestamp)] = { - val originalTsStrings = 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" - ) - val rowRdd = spark.sparkContext.parallelize(originalTsStrings, 1).map(Row(_)) + val rowRdd = spark.sparkContext.parallelize(desiredTimestampStrings, 1).map(Row(_)) val schema = StructType(Seq( StructField("display", StringType, true) )) @@ -300,9 +294,7 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi // this will get the millis corresponding to the display time given the current *session* // timezone. import spark.implicits._ - df.withColumn("ts", expr("cast(display as timestamp)")).map { row => - (row.getAs[String](0), row.getAs[Timestamp](1)) - } + df.withColumn("ts", expr("cast(display as timestamp)")).as[(String, Timestamp)] } private def testWriteTablesWithTimezone( @@ -349,15 +341,20 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi // values in the parquet file. val onDiskLocation = spark.sessionState.catalog .getTableMetadata(TableIdentifier(s"insert_$baseTable")).location.getPath - val readFromDisk = spark.read.parquet(onDiskLocation).collect() - val storageTzId = explicitTz.getOrElse(sessionTzId) - readFromDisk.foreach { row => - val displayTime = row.getAs[String](0) - val millis = row.getAs[Timestamp](1).getTime() - val expectedMillis = timestampTimezoneToMillis((displayTime, storageTzId)) - assert(expectedMillis === millis, s"Display time '$displayTime' was stored incorrectly " + - s"with sessionTz = ${sessionTzOpt}; Got $millis, expected $expectedMillis " + - s"(delta = ${millis - expectedMillis})") + // we test reading the data back with and without the vectorized reader, to make sure we + // haven't broken reading parquet from non-hive tables, with both readers. + Seq(false, true).foreach { vectorized => + spark.conf.set(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, vectorized) + val readFromDisk = spark.read.parquet(onDiskLocation).collect() + val storageTzId = explicitTz.getOrElse(sessionTzId) + readFromDisk.foreach { row => + val displayTime = row.getAs[String](0) + val millis = row.getAs[Timestamp](1).getTime() + val expectedMillis = timestampTimezoneToMillis((displayTime, storageTzId)) + assert(expectedMillis === millis, s"Display time '$displayTime' was stored " + + s"incorrectly with sessionTz = ${sessionTzOpt}; Got $millis, expected " + + s"$expectedMillis (delta = ${millis - expectedMillis})") + } } } } @@ -401,65 +398,69 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi options = options ) Seq(false, true).foreach { vectorized => - withSQLConf((SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, vectorized.toString)) { - withClue(s"vectorized = $vectorized;") { - val sessionTz = sessionTzOpt.getOrElse(TimeZone.getDefault().getID()) - val collectedFromExternal = - spark.sql(s"select display, ts from external_$baseTable").collect() - collectedFromExternal.foreach { row => - val displayTime = row.getAs[String](0) - val millis = row.getAs[Timestamp](1).getTime() - val expectedMillis = timestampTimezoneToMillis((displayTime, sessionTz)) - val delta = millis - expectedMillis - val deltaHours = delta / (1000L * 60 * 60) - assert(millis === expectedMillis, s"Display time '$displayTime' did not have " + - s"correct millis: was $millis, expected $expectedMillis; delta = $delta " + - s"($deltaHours hours)") - } - - // Now test that the behavior is still correct even with a filter which could get - // pushed down into parquet. We don't need extra handling for pushed down - // predicates because (a) in ParquetFilters, we ignore TimestampType and (b) parquet - // does not read statistics from int96 fields, as they are unsigned. See - // scalastyle:off line.size.limit - // https://github.com/apache/parquet-mr/blob/2fd62ee4d524c270764e9b91dca72e5cf1a005b7/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java#L419 - // https://github.com/apache/parquet-mr/blob/2fd62ee4d524c270764e9b91dca72e5cf1a005b7/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java#L348 - // scalastyle:on line.size.limit - // - // Just to be defensive in case anything ever changes in parquet, this test checks - // the assumption on column stats, and also the end-to-end behavior. - - val hadoopConf = sparkContext.hadoopConfiguration - val fs = FileSystem.get(hadoopConf) - val parts = fs.listStatus(new Path(path.getCanonicalPath)) - .filter(_.getPath().getName().endsWith(".parquet")) - // grab the meta data from the parquet file. The next section of asserts just make - // sure the test is configured correctly. - assert(parts.size == 1) - val oneFooter = ParquetFileReader.readFooter(hadoopConf, parts.head.getPath) - assert(oneFooter.getFileMetaData.getSchema.getColumns.size === 2) - assert(oneFooter.getFileMetaData.getSchema.getColumns.get(1).getType() === - PrimitiveTypeName.INT96) - val oneBlockMeta = oneFooter.getBlocks().get(0) - val oneBlockColumnMeta = oneBlockMeta.getColumns().get(1) - val columnStats = oneBlockColumnMeta.getStatistics - // This is the important assert. Column stats are written, but they are ignored - // when the data is read back as mentioned above, b/c int96 is unsigned. This - // assert makes sure this holds even if we change parquet versions (if eg. there - // were ever statistics even on unsigned columns). - assert(columnStats.isEmpty) - - // 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 external_$baseTable where ts $comparison '$value'" - val countWithFilter = spark.sql(query).count() - assert(countWithFilter === 4, query) - } + withClue(s"vectorized = $vectorized;") { + spark.conf.set(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, vectorized) + val sessionTz = sessionTzOpt.getOrElse(TimeZone.getDefault().getID()) + val query = s"select display, cast(ts as string) as ts_as_string, ts " + + s"from external_$baseTable" + val collectedFromExternal = spark.sql(query).collect() + collectedFromExternal.foreach { row => + val displayTime = row.getAs[String](0) + // the timestamp should still display the same, despite the changes in timezones + assert(displayTime === row.getAs[String](1).toString()) + // we'll also check that the millis behind the timestamp has the appropriate + // adjustments. + val millis = row.getAs[Timestamp](2).getTime() + val expectedMillis = timestampTimezoneToMillis((displayTime, sessionTz)) + val delta = millis - expectedMillis + val deltaHours = delta / (1000L * 60 * 60) + assert(millis === expectedMillis, s"Display time '$displayTime' did not have " + + s"correct millis: was $millis, expected $expectedMillis; delta = $delta " + + s"($deltaHours hours)") + } + + // Now test that the behavior is still correct even with a filter which could get + // pushed down into parquet. We don't need extra handling for pushed down + // predicates because (a) in ParquetFilters, we ignore TimestampType and (b) parquet + // does not read statistics from int96 fields, as they are unsigned. See + // scalastyle:off line.size.limit + // https://github.com/apache/parquet-mr/blob/2fd62ee4d524c270764e9b91dca72e5cf1a005b7/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java#L419 + // https://github.com/apache/parquet-mr/blob/2fd62ee4d524c270764e9b91dca72e5cf1a005b7/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java#L348 + // scalastyle:on line.size.limit + // + // Just to be defensive in case anything ever changes in parquet, this test checks + // the assumption on column stats, and also the end-to-end behavior. + + val hadoopConf = sparkContext.hadoopConfiguration + val fs = FileSystem.get(hadoopConf) + val parts = fs.listStatus(new Path(path.getCanonicalPath)) + .filter(_.getPath().getName().endsWith(".parquet")) + // grab the meta data from the parquet file. The next section of asserts just make + // sure the test is configured correctly. + assert(parts.size == 1) + val oneFooter = ParquetFileReader.readFooter(hadoopConf, parts.head.getPath) + assert(oneFooter.getFileMetaData.getSchema.getColumns.size === 2) + assert(oneFooter.getFileMetaData.getSchema.getColumns.get(1).getType() === + PrimitiveTypeName.INT96) + val oneBlockMeta = oneFooter.getBlocks().get(0) + val oneBlockColumnMeta = oneBlockMeta.getColumns().get(1) + val columnStats = oneBlockColumnMeta.getStatistics + // This is the important assert. Column stats are written, but they are ignored + // when the data is read back as mentioned above, b/c int96 is unsigned. This + // assert makes sure this holds even if we change parquet versions (if eg. there + // were ever statistics even on unsigned columns). + assert(columnStats.isEmpty) + + // 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 external_$baseTable where ts $comparison '$value'" + val countWithFilter = spark.sql(query).count() + assert(countWithFilter === 4, query) } } } From 44a8bbb17484a61dd984fcb451e3b1be8c539e9f Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 19 Apr 2017 23:11:05 -0500 Subject: [PATCH 35/40] better param names and docs --- .../parquet/ParquetRowConverter.scala | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala index 3cdd1b882886..2f9a7329c602 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala @@ -275,8 +275,8 @@ private[parquet] class ParquetRowConverter( new ParquetPrimitiveConverter(updater) { // Converts nanosecond timestamps stored as INT96 override def addBinary(value: Binary): Unit = { - val timestamp = ParquetRowConverter.binaryToSQLTimestamp(value, fromTz = sessionTz, - toTz = storageTz) + val timestamp = ParquetRowConverter.binaryToSQLTimestamp(value, sessionTz = sessionTz, + storageTz = storageTz) updater.setLong(timestamp) } } @@ -684,10 +684,19 @@ private[parquet] object ParquetRowConverter { * The timestamp is really meant to be interpreted as a "floating time", but since we * actually store it as micros since epoch, why we have to apply a conversion when timezones * change. + * * @param binary + * @param sessionTz the session timezone. This will be used to determine how to display the time, + * and compute functions on the timestamp which involve a timezone, eg. extract + * the hour. + * @param storageTz the timezone which was used to store the timestamp. This should come from the + * timestamp table property, or else assume its the same as the sessionTz * @return */ - def binaryToSQLTimestamp(binary: Binary, fromTz: TimeZone, toTz: TimeZone): SQLTimestamp = { + def binaryToSQLTimestamp( + binary: Binary, + sessionTz: TimeZone, + storageTz: TimeZone): SQLTimestamp = { assert(binary.length() == 12, s"Timestamps (with nanoseconds) are expected to be stored in" + s" 12-byte long binaries. Found a ${binary.length()}-byte binary instead.") val buffer = binary.toByteBuffer.order(ByteOrder.LITTLE_ENDIAN) @@ -695,8 +704,8 @@ private[parquet] object ParquetRowConverter { val julianDay = buffer.getInt val utcEpochMicros = DateTimeUtils.fromJulianDay(julianDay, timeOfDayNanos) // avoid expensive time logic if possible. - if (fromTz.getID() != toTz.getID()) { - DateTimeUtils.convertTz(utcEpochMicros, fromTz, toTz) + if (sessionTz.getID() != storageTz.getID()) { + DateTimeUtils.convertTz(utcEpochMicros, sessionTz, storageTz) } else { utcEpochMicros } From e31657a1b65ab73d52651c4e0b018d457e44d47f Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 25 Apr 2017 11:46:39 -0500 Subject: [PATCH 36/40] review feedback --- .../parquet/ParquetRowConverter.scala | 8 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 6 +- .../hive/ParquetHiveCompatibilitySuite.scala | 147 +++++++++++------- 3 files changed, 92 insertions(+), 69 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala index 2f9a7329c602..d52ff62d93b2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala @@ -267,8 +267,8 @@ private[parquet] class ParquetRowConverter( case TimestampType => // TODO Implements `TIMESTAMP_MICROS` once parquet-mr has that. // If the table has a timezone property, apply the correct conversions. See SPARK-12297. - val sessionTsString = hadoopConf.get(SQLConf.SESSION_LOCAL_TIMEZONE.key) - val sessionTz = Option(sessionTsString).map(TimeZone.getTimeZone(_)) + val sessionTzString = hadoopConf.get(SQLConf.SESSION_LOCAL_TIMEZONE.key) + val sessionTz = Option(sessionTzString).map(TimeZone.getTimeZone(_)) .getOrElse(DateTimeUtils.defaultTimeZone()) val storageTzString = hadoopConf.get(ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY) val storageTz = Option(storageTzString).map(TimeZone.getTimeZone(_)).getOrElse(sessionTz) @@ -685,13 +685,13 @@ private[parquet] object ParquetRowConverter { * actually store it as micros since epoch, why we have to apply a conversion when timezones * change. * - * @param binary + * @param binary a parquet Binary which holds one int96 * @param sessionTz the session timezone. This will be used to determine how to display the time, * and compute functions on the timestamp which involve a timezone, eg. extract * the hour. * @param storageTz the timezone which was used to store the timestamp. This should come from the * timestamp table property, or else assume its the same as the sessionTz - * @return + * @return a timestamp (millis since epoch) which will render correctly in the sessionTz */ def binaryToSQLTimestamp( binary: Binary, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 490bce11c6a8..99171195453b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -228,11 +228,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log // We add the table timezone to the relation options, which automatically gets injected into the // hadoopConf for the Parquet Converters val storageTzKey = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY - val storageTz = relation.tableMeta.properties.getOrElse(storageTzKey, "") - val sessionTz = sparkSession.sessionState.conf.sessionLocalTimeZone - Map( - storageTzKey -> storageTz - ) + relation.tableMeta.properties.get(storageTzKey).map(storageTzKey -> _).toMap } private def inferIfNeeded( 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 1d0309db42ca..ef5dff31c84a 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,6 +17,7 @@ package org.apache.spark.sql.hive +import java.io.File import java.sql.Timestamp import java.util.TimeZone @@ -52,15 +53,6 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi """.stripMargin) } - override def afterEach(): Unit = { - try { - // drop all databases, tables and functions after each test - spark.sessionState.catalog.reset() - } finally { - super.afterEach() - } - } - private def testParquetHiveCompatibility(row: Row, hiveTypes: String*): Unit = { withTable("parquet_compat") { withTempPath { dir => @@ -180,11 +172,7 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi testCreateWriteRead(sparkSession, "local", Some(localTz), sessionTzOpt) // check with a variety of timezones. The unit tests currently are configured to always use // America/Los_Angeles, but even if they didn't, we'd be sure to cover a non-local timezone. - Seq( - "UTC" -> "UTC", - "LA" -> "America/Los_Angeles", - "Berlin" -> "Europe/Berlin" - ).foreach { case (tableName, zone) => + testTimezones.foreach { case (tableName, zone) => if (zone != localTz) { testCreateWriteRead(sparkSession, tableName, Some(zone), sessionTzOpt) } @@ -201,7 +189,7 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi testReadTablesWithTimezone(sparkSession, baseTable, explicitTz, sessionTzOpt) } - private def checkHasTz(table: String, tz: Option[String]): Unit = { + 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) } @@ -214,9 +202,7 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi test(s"SPARK-12297: Create and Alter Parquet tables and timezones; explicitTz = $explicitTz; " + s"sessionTzOpt = $sessionTzOpt") { val key = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY - withTable(baseTable, s"like_$baseTable", s"select_$baseTable") { - val localTz = TimeZone.getDefault() - val localTzId = localTz.getID() + 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 @@ -231,28 +217,37 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi | $tblProperties """.stripMargin) val expectedTableTz = explicitTz.orElse(defaultTz) - checkHasTz(baseTable, expectedTableTz) + checkHasTz(spark, baseTable, expectedTableTz) + spark.sql( + raw"""CREATE TABLE partitioned_$baseTable ( + | x int + | ) + | PARTITIONED BY (y int) + | STORED AS PARQUET + | $tblProperties + """.stripMargin) + checkHasTz(spark, s"partitioned_$baseTable", expectedTableTz) spark.sql(s"CREATE TABLE like_$baseTable LIKE $baseTable") - checkHasTz(s"like_$baseTable", expectedTableTz) + checkHasTz(spark, s"like_$baseTable", expectedTableTz) spark.sql( raw"""CREATE TABLE select_$baseTable | STORED AS PARQUET | AS | SELECT * from $baseTable """.stripMargin) - checkHasTz(s"select_$baseTable", defaultTz) + checkHasTz(spark, s"select_$baseTable", defaultTz) // check alter table, setting, unsetting, resetting the property spark.sql( raw"""ALTER TABLE $baseTable SET TBLPROPERTIES ($key="America/Los_Angeles")""") - checkHasTz(baseTable, Some("America/Los_Angeles")) - spark.sql( raw"""ALTER TABLE $baseTable SET TBLPROPERTIES ($key="UTC")""") - checkHasTz(baseTable, Some("UTC")) - spark.sql( raw"""ALTER TABLE $baseTable UNSET TBLPROPERTIES ($key)""") - checkHasTz(baseTable, None) + checkHasTz(spark, baseTable, Some("America/Los_Angeles")) + spark.sql(raw"""ALTER TABLE $baseTable SET TBLPROPERTIES ($key="UTC")""") + checkHasTz(spark, baseTable, Some("UTC")) + spark.sql(raw"""ALTER TABLE $baseTable UNSET TBLPROPERTIES ($key)""") + checkHasTz(spark, baseTable, None) explicitTz.foreach { tz => - spark.sql( raw"""ALTER TABLE $baseTable SET TBLPROPERTIES ($key="$tz")""") - checkHasTz(baseTable, expectedTableTz) + spark.sql(raw"""ALTER TABLE $baseTable SET TBLPROPERTIES ($key="$tz")""") + checkHasTz(spark, baseTable, expectedTableTz) } } } @@ -269,31 +264,29 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi // 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 originalTz = TimeZone.getDefault - val timestampTimezoneToMillis = try { - (for { - timestampString <- desiredTimestampStrings - timezone <- Seq("America/Los_Angeles", "Europe/Berlin", "UTC").map { - TimeZone.getTimeZone(_) - } - } yield { - TimeZone.setDefault(timezone) - val timestamp = Timestamp.valueOf(timestampString) - (timestampString, timezone.getID()) -> timestamp.getTime() - }).toMap - } finally { - TimeZone.setDefault(originalTz) + val timestampTimezoneToMillis = { + val originalTz = TimeZone.getDefault + try { + (for { + timestampString <- desiredTimestampStrings + timezone <- Seq("America/Los_Angeles", "Europe/Berlin", "UTC").map { + TimeZone.getTimeZone(_) + } + } yield { + TimeZone.setDefault(timezone) + val timestamp = Timestamp.valueOf(timestampString) + (timestampString, timezone.getID()) -> timestamp.getTime() + }).toMap + } finally { + TimeZone.setDefault(originalTz) + } } private def createRawData(spark: SparkSession): Dataset[(String, Timestamp)] = { - val rowRdd = spark.sparkContext.parallelize(desiredTimestampStrings, 1).map(Row(_)) - val schema = StructType(Seq( - StructField("display", StringType, true) - )) - val df = spark.createDataFrame(rowRdd, schema) + import spark.implicits._ + val df = desiredTimestampStrings.toDF("display") // this will get the millis corresponding to the display time given the current *session* // timezone. - import spark.implicits._ df.withColumn("ts", expr("cast(display as timestamp)")).as[(String, Timestamp)] } @@ -319,7 +312,7 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi // We write data into our tables, and then check the raw parquet files to see whether // the correct conversion was applied. rawData.write.saveAsTable(s"saveAsTable_$baseTable") - checkHasTz(s"saveAsTable_$baseTable", None) + checkHasTz(spark, s"saveAsTable_$baseTable", None) spark.sql( raw"""CREATE TABLE insert_$baseTable ( | display string, @@ -328,7 +321,7 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi | STORED AS PARQUET | $tblProperties """.stripMargin) - checkHasTz(s"insert_$baseTable", explicitTz) + checkHasTz(spark, s"insert_$baseTable", explicitTz) rawData.write.insertInto(s"insert_$baseTable") // no matter what, roundtripping via the table should leave the data unchanged val readFromTable = spark.table(s"insert_$baseTable").collect() @@ -368,7 +361,7 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi val key = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY test(s"SPARK-12297: Read from Parquet tables with Timestamps; explicitTz = $explicitTz; " + s"sessionTzOpt = $sessionTzOpt") { - withTable(s"external_$baseTable") { + withTable(s"external_$baseTable", s"partitioned_$baseTable") { // we intentionally save this data directly, without creating a table, so we can // see that the data is read back differently depending on table properties. // we'll save with adjusted millis, so that it should be the correct millis after reading @@ -386,9 +379,11 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi case _ => rawData }).withColumnRenamed("_1", "display").withColumnRenamed("_2", "ts") - withTempPath { path => - adjustedRawData.write.parquet(path.getCanonicalPath) - val options = Map("path" -> path.getCanonicalPath) ++ + withTempPath { basePath => + val unpartitionedPath = new File(basePath, "flat") + val partitionedPath = new File(basePath, "partitioned") + adjustedRawData.write.parquet(unpartitionedPath.getCanonicalPath) + val options = Map("path" -> unpartitionedPath.getCanonicalPath) ++ explicitTz.map { tz => Map(key -> tz) }.getOrElse(Map()) spark.catalog.createTable( @@ -397,13 +392,38 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi schema = new StructType().add("display", StringType).add("ts", TimestampType), options = options ) - Seq(false, true).foreach { vectorized => - withClue(s"vectorized = $vectorized;") { + + // also write out a partitioned table, to make sure we can access that correctly. + // add a column we can partition by (value doesn't particularly matter). + val partitionedData = adjustedRawData.withColumn("id", monotonicallyIncreasingId) + partitionedData.write.partitionBy("id") + .parquet(partitionedPath.getCanonicalPath) + // unfortunately, catalog.createTable() doesn't let us specify partitioning, so just use + // a "CREATE TABLE" stmt. + val tblOpts = explicitTz.map { tz => raw"""TBLPROPERTIES ($key="$tz")""" }.getOrElse("") + spark.sql(raw"""CREATE EXTERNAL TABLE partitioned_$baseTable ( + | display string, + | ts timestamp + |) + |PARTITIONED BY (id bigint) + |STORED AS parquet + |LOCATION 'file:${partitionedPath.getCanonicalPath}' + |$tblOpts + """.stripMargin) + spark.sql(s"msck repair table partitioned_$baseTable") + + for { + vectorized <- Seq(false, true) + partitioned <- Seq(false, true) + } { + withClue(s"vectorized = $vectorized; partitioned = $partitioned") { spark.conf.set(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, vectorized) val sessionTz = sessionTzOpt.getOrElse(TimeZone.getDefault().getID()) + val table = if (partitioned) s"partitioned_$baseTable" else s"external_$baseTable" val query = s"select display, cast(ts as string) as ts_as_string, ts " + - s"from external_$baseTable" + s"from $table" val collectedFromExternal = spark.sql(query).collect() + assert( collectedFromExternal.size === 4) collectedFromExternal.foreach { row => val displayTime = row.getAs[String](0) // the timestamp should still display the same, despite the changes in timezones @@ -433,8 +453,15 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi val hadoopConf = sparkContext.hadoopConfiguration val fs = FileSystem.get(hadoopConf) - val parts = fs.listStatus(new Path(path.getCanonicalPath)) - .filter(_.getPath().getName().endsWith(".parquet")) + val parts = if (partitioned) { + val subdirs = fs.listStatus(new Path(partitionedPath.getCanonicalPath)) + .filter(_.getPath().getName().startsWith("id=")) + fs.listStatus(subdirs.head.getPath()) + .filter(_.getPath().getName().endsWith(".parquet")) + } else { + fs.listStatus(new Path(unpartitionedPath.getCanonicalPath)) + .filter(_.getPath().getName().endsWith(".parquet")) + } // grab the meta data from the parquet file. The next section of asserts just make // sure the test is configured correctly. assert(parts.size == 1) @@ -458,7 +485,7 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi "<" -> "2016-01-01 02:00:00" ).foreach { case (comparison, value) => val query = - s"select ts from external_$baseTable where ts $comparison '$value'" + s"select ts from $table where ts $comparison '$value'" val countWithFilter = spark.sql(query).count() assert(countWithFilter === 4, query) } From acc72eaaec19d0fb040a02a224df5c0ac8d3be38 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 2 May 2017 13:41:18 -0500 Subject: [PATCH 37/40] add check for partitioned tables --- .../hive/ParquetHiveCompatibilitySuite.scala | 17 +++++++++++++++-- 1 file changed, 15 insertions(+), 2 deletions(-) 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 ef5dff31c84a..7c0838d90865 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 @@ -18,6 +18,7 @@ package org.apache.spark.sql.hive import java.io.File +import java.net.URLDecoder import java.sql.Timestamp import java.util.TimeZone @@ -32,7 +33,7 @@ import org.apache.spark.sql.execution.datasources.parquet.{ParquetCompatibilityT 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, StructField, StructType, TimestampType} +import org.apache.spark.sql.types.{StringType, StructType, TimestampType} class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHiveSingleton with BeforeAndAfterEach { @@ -299,7 +300,7 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi test(s"SPARK-12297: Write to Parquet tables with Timestamps; explicitTz = $explicitTz; " + s"sessionTzOpt = $sessionTzOpt") { - withTable(s"saveAsTable_$baseTable", s"insert_$baseTable") { + withTable(s"saveAsTable_$baseTable", s"insert_$baseTable", s"partitioned_ts_$baseTable") { val sessionTzId = sessionTzOpt.getOrElse(TimeZone.getDefault().getID()) // check that created tables have correct TBLPROPERTIES val tblProperties = explicitTz.map { @@ -349,6 +350,18 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi s"$expectedMillis (delta = ${millis - expectedMillis})") } } + + // check tables partitioned by timestamps. We don't compare the "raw" data in this case, + // since they are adjusted even when we bypass the hive table. + rawData.write.partitionBy("ts").saveAsTable(s"partitioned_ts_$baseTable") + val partitionDiskLocation = spark.sessionState.catalog + .getTableMetadata(TableIdentifier(s"partitioned_ts_$baseTable")).location.getPath + // no matter what mix of timezones we use, the dirs should specify the value with the + // same time we use for display. + val parts = new File(partitionDiskLocation).list().collect { + case name if name.startsWith("ts=") => URLDecoder.decode(name.stripPrefix("ts=")) + }.toSet + assert(parts === desiredTimestampStrings.toSet) } } } From b9c03e963177729a0fb6afeb09ddf8fdda18072b Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 2 May 2017 13:42:51 -0500 Subject: [PATCH 38/40] fix typo --- .../org/apache/spark/sql/catalyst/catalog/interface.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index cc0cbba275b8..c39017ebbfe6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -132,10 +132,10 @@ case class CatalogTablePartition( /** * Given the partition schema, returns a row with that schema holding the partition values. */ - def toRow(partitionSchema: StructType, defaultTimeZondId: String): InternalRow = { + def toRow(partitionSchema: StructType, defaultTimeZoneId: String): InternalRow = { val caseInsensitiveProperties = CaseInsensitiveMap(storage.properties) val timeZoneId = caseInsensitiveProperties.getOrElse( - DateTimeUtils.TIMEZONE_OPTION, defaultTimeZondId) + DateTimeUtils.TIMEZONE_OPTION, defaultTimeZoneId) InternalRow.fromSeq(partitionSchema.map { field => val partValue = if (spec(field.name) == ExternalCatalogUtils.DEFAULT_PARTITION_NAME) { null From fc17a2edc8e46dd95019f8ece972610d40f3ed2c Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 2 May 2017 13:44:43 -0500 Subject: [PATCH 39/40] review feedback --- .../apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) 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 7c0838d90865..bf3fe6b8c06c 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 @@ -35,8 +35,7 @@ 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 - with BeforeAndAfterEach { +class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHiveSingleton { /** * Set the staging directory (and hence path to ignore Parquet files under) * to the default value of hive.exec.stagingdir. From 2537437a0c1f022acc215518e4728d24f6f6cf97 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 3 May 2017 10:21:23 -0500 Subject: [PATCH 40/40] review feedback --- .../spark/sql/hive/HiveMetastoreCatalog.scala | 1 - .../hive/ParquetHiveCompatibilitySuite.scala | 45 +++++++++---------- 2 files changed, 20 insertions(+), 26 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 99171195453b..e0b565c0d79a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -30,7 +30,6 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode._ import org.apache.spark.sql.types._ 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 bf3fe6b8c06c..2bfd63d9b56e 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 @@ -25,7 +25,6 @@ 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.scalatest.BeforeAndAfterEach import org.apache.spark.sql.{AnalysisException, Dataset, Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier @@ -207,10 +206,10 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi val defaultTz = None // check that created tables have correct TBLPROPERTIES val tblProperties = explicitTz.map { - tz => raw"""TBLPROPERTIES ($key="$tz")""" + tz => s"""TBLPROPERTIES ($key="$tz")""" }.getOrElse("") spark.sql( - raw"""CREATE TABLE $baseTable ( + s"""CREATE TABLE $baseTable ( | x int | ) | STORED AS PARQUET @@ -219,7 +218,7 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi val expectedTableTz = explicitTz.orElse(defaultTz) checkHasTz(spark, baseTable, expectedTableTz) spark.sql( - raw"""CREATE TABLE partitioned_$baseTable ( + s"""CREATE TABLE partitioned_$baseTable ( | x int | ) | PARTITIONED BY (y int) @@ -230,7 +229,7 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi spark.sql(s"CREATE TABLE like_$baseTable LIKE $baseTable") checkHasTz(spark, s"like_$baseTable", expectedTableTz) spark.sql( - raw"""CREATE TABLE select_$baseTable + s"""CREATE TABLE select_$baseTable | STORED AS PARQUET | AS | SELECT * from $baseTable @@ -239,14 +238,14 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi // check alter table, setting, unsetting, resetting the property spark.sql( - raw"""ALTER TABLE $baseTable SET TBLPROPERTIES ($key="America/Los_Angeles")""") + s"""ALTER TABLE $baseTable SET TBLPROPERTIES ($key="America/Los_Angeles")""") checkHasTz(spark, baseTable, Some("America/Los_Angeles")) - spark.sql(raw"""ALTER TABLE $baseTable SET TBLPROPERTIES ($key="UTC")""") + spark.sql(s"""ALTER TABLE $baseTable SET TBLPROPERTIES ($key="UTC")""") checkHasTz(spark, baseTable, Some("UTC")) - spark.sql(raw"""ALTER TABLE $baseTable UNSET TBLPROPERTIES ($key)""") + spark.sql(s"""ALTER TABLE $baseTable UNSET TBLPROPERTIES ($key)""") checkHasTz(spark, baseTable, None) explicitTz.foreach { tz => - spark.sql(raw"""ALTER TABLE $baseTable SET TBLPROPERTIES ($key="$tz")""") + spark.sql(s"""ALTER TABLE $baseTable SET TBLPROPERTIES ($key="$tz")""") checkHasTz(spark, baseTable, expectedTableTz) } } @@ -267,16 +266,13 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi val timestampTimezoneToMillis = { val originalTz = TimeZone.getDefault try { - (for { - timestampString <- desiredTimestampStrings - timezone <- Seq("America/Los_Angeles", "Europe/Berlin", "UTC").map { - TimeZone.getTimeZone(_) + 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() } - } yield { - TimeZone.setDefault(timezone) - val timestamp = Timestamp.valueOf(timestampString) - (timestampString, timezone.getID()) -> timestamp.getTime() - }).toMap + }.toMap } finally { TimeZone.setDefault(originalTz) } @@ -303,10 +299,9 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi val sessionTzId = sessionTzOpt.getOrElse(TimeZone.getDefault().getID()) // check that created tables have correct TBLPROPERTIES val tblProperties = explicitTz.map { - tz => raw"""TBLPROPERTIES ($key="$tz")""" + tz => s"""TBLPROPERTIES ($key="$tz")""" }.getOrElse("") - val rawData = createRawData(spark) // Check writing data out. // We write data into our tables, and then check the raw parquet files to see whether @@ -314,7 +309,7 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi rawData.write.saveAsTable(s"saveAsTable_$baseTable") checkHasTz(spark, s"saveAsTable_$baseTable", None) spark.sql( - raw"""CREATE TABLE insert_$baseTable ( + s"""CREATE TABLE insert_$baseTable ( | display string, | ts timestamp | ) @@ -370,7 +365,7 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi baseTable: String, explicitTz: Option[String], sessionTzOpt: Option[String]): Unit = { - val key = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY + val key = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY test(s"SPARK-12297: Read from Parquet tables with Timestamps; explicitTz = $explicitTz; " + s"sessionTzOpt = $sessionTzOpt") { withTable(s"external_$baseTable", s"partitioned_$baseTable") { @@ -412,8 +407,8 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi .parquet(partitionedPath.getCanonicalPath) // unfortunately, catalog.createTable() doesn't let us specify partitioning, so just use // a "CREATE TABLE" stmt. - val tblOpts = explicitTz.map { tz => raw"""TBLPROPERTIES ($key="$tz")""" }.getOrElse("") - spark.sql(raw"""CREATE EXTERNAL TABLE partitioned_$baseTable ( + val tblOpts = explicitTz.map { tz => s"""TBLPROPERTIES ($key="$tz")""" }.getOrElse("") + spark.sql(s"""CREATE EXTERNAL TABLE partitioned_$baseTable ( | display string, | ts timestamp |) @@ -512,7 +507,7 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi val key = ParquetFileFormat.PARQUET_TIMEZONE_TABLE_PROPERTY val badTzException = intercept[AnalysisException] { spark.sql( - raw"""CREATE TABLE bad_tz_table ( + s"""CREATE TABLE bad_tz_table ( | x int | ) | STORED AS PARQUET