[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...
Github user asfgit closed the pull request at: https://github.com/apache/spark/pull/21935 --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...
Github user gengliangwang commented on a diff in the pull request: https://github.com/apache/spark/pull/21935#discussion_r207159637 --- Diff: external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala --- @@ -41,6 +41,7 @@ import org.apache.spark.sql.types._ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { val episodesAvro = testFile("episodes.avro") val testAvro = testFile("test.avro") + val timestampAvro = testFile("timestamp.avro") --- End diff -- The schema and data is stated in https://github.com/apache/spark/pull/21935/files#diff-9364b0610f92b3cc35a4bc43a80751bfR397 It should be easy to get from test cases. The other test file episodesAvro also doesn't provide how it is generated. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/21935#discussion_r207119879 --- Diff: external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala --- @@ -41,6 +41,7 @@ import org.apache.spark.sql.types._ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { val episodesAvro = testFile("episodes.avro") val testAvro = testFile("test.avro") + val timestampAvro = testFile("timestamp.avro") --- End diff -- at least we should provide how the binary file is generated, or just do roundtrip test: Spark write avro files and then read it. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...
Github user gengliangwang commented on a diff in the pull request: https://github.com/apache/spark/pull/21935#discussion_r207110511 --- Diff: external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala --- @@ -103,31 +108,49 @@ object SchemaConverters { catalystType: DataType, nullable: Boolean = false, recordName: String = "topLevelRecord", - prevNameSpace: String = ""): Schema = { + prevNameSpace: String = "", + outputTimestampType: AvroOutputTimestampType.Value = AvroOutputTimestampType.TIMESTAMP_MICROS --- End diff -- It is also used in `CatalystDataToAvro` --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...
Github user HyukjinKwon commented on a diff in the pull request: https://github.com/apache/spark/pull/21935#discussion_r207100964 --- Diff: external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala --- @@ -86,8 +87,16 @@ class AvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType) { case (LONG, LongType) => (updater, ordinal, value) => updater.setLong(ordinal, value.asInstanceOf[Long]) - case (LONG, TimestampType) => (updater, ordinal, value) => -updater.setLong(ordinal, value.asInstanceOf[Long] * 1000) + case (LONG, TimestampType) => avroType.getLogicalType match { +case _: TimestampMillis => (updater, ordinal, value) => + updater.setLong(ordinal, value.asInstanceOf[Long] * 1000) +case _: TimestampMicros => (updater, ordinal, value) => + updater.setLong(ordinal, value.asInstanceOf[Long]) +case null => (updater, ordinal, value) => --- End diff -- ditto, add a default case. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/21935#discussion_r207095659 --- Diff: external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala --- @@ -103,31 +108,49 @@ object SchemaConverters { catalystType: DataType, nullable: Boolean = false, recordName: String = "topLevelRecord", - prevNameSpace: String = ""): Schema = { + prevNameSpace: String = "", + outputTimestampType: AvroOutputTimestampType.Value = AvroOutputTimestampType.TIMESTAMP_MICROS --- End diff -- do we really need the default value? Seems only one call site excluding the recursive ones. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/21935#discussion_r207095392 --- Diff: external/avro/src/main/scala/org/apache/spark/sql/avro/AvroSerializer.scala --- @@ -93,7 +94,13 @@ class AvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: case DateType => (getter, ordinal) => getter.getInt(ordinal) * DateTimeUtils.MILLIS_PER_DAY case TimestampType => -(getter, ordinal) => getter.getLong(ordinal) / 1000 +(getter, ordinal) => avroType.getLogicalType match { + case _: TimestampMillis => getter.getLong(ordinal) / 1000 + case _: TimestampMicros => getter.getLong(ordinal) + // For backward compatibility, if the Avro type is Long and it is not logical type, + // output the timestamp value as with millisecond precision. + case null => getter.getLong(ordinal) / 1000 --- End diff -- ditto, add a default case. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/21935#discussion_r207095253 --- Diff: external/avro/src/main/scala/org/apache/spark/sql/avro/AvroSerializer.scala --- @@ -93,7 +94,13 @@ class AvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: case DateType => (getter, ordinal) => getter.getInt(ordinal) * DateTimeUtils.MILLIS_PER_DAY case TimestampType => -(getter, ordinal) => getter.getLong(ordinal) / 1000 +(getter, ordinal) => avroType.getLogicalType match { --- End diff -- do not do pattern match per record, we should ``` avroType.getLogicalType match { case _: TimestampMillis => (getter, ordinal) => ... ``` --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/21935#discussion_r207095177 --- Diff: external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOptions.scala --- @@ -79,4 +80,16 @@ class AvroOptions( val compression: String = { parameters.get("compression").getOrElse(SQLConf.get.avroCompressionCodec) } + + /** + * The `outputTimestampType` option sets which Avro timestamp type to use when Spark writes + * data to Avro files. Currently supported types are `TIMESTAMP_MICROS` and `TIMESTAMP_MILLIS`. + * TIMESTAMP_MICROS is a logical timestamp type in Avro, which stores number of microseconds + * from the Unix epoch. TIMESTAMP_MILLIS is also logical, but with millisecond precision, + * which means Spark has to truncate the microsecond portion of its timestamp value. + */ + val outputTimestampType: AvroOutputTimestampType.Value = { --- End diff -- I'm ok with it, I think parquet should also follow this. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/21935#discussion_r207094996 --- Diff: external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala --- @@ -86,8 +87,16 @@ class AvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType) { case (LONG, LongType) => (updater, ordinal, value) => updater.setLong(ordinal, value.asInstanceOf[Long]) - case (LONG, TimestampType) => (updater, ordinal, value) => -updater.setLong(ordinal, value.asInstanceOf[Long] * 1000) + case (LONG, TimestampType) => avroType.getLogicalType match { +case _: TimestampMillis => (updater, ordinal, value) => + updater.setLong(ordinal, value.asInstanceOf[Long] * 1000) +case _: TimestampMicros => (updater, ordinal, value) => + updater.setLong(ordinal, value.asInstanceOf[Long]) +case null => (updater, ordinal, value) => + // For backward compatibility, if the Avro type is Long and it is not logical type, + // the value is processed as timestamp type with millisecond precision. + updater.setLong(ordinal, value.asInstanceOf[Long] * 1000) + } --- End diff -- we should add a default case and throw IncompatibleSchemaException, in case avro add more logical types for long type in the future. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...
Github user HyukjinKwon commented on a diff in the pull request: https://github.com/apache/spark/pull/21935#discussion_r207089232 --- Diff: external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala --- @@ -36,11 +36,12 @@ import org.apache.spark.sql._ import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} -import org.apache.spark.sql.types._ +import org.apache.spark.sql.types.{StructType, _} --- End diff -- Import looks a bit odd :-) --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...
Github user HyukjinKwon commented on a diff in the pull request: https://github.com/apache/spark/pull/21935#discussion_r207089217 --- Diff: external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala --- @@ -103,31 +108,49 @@ object SchemaConverters { catalystType: DataType, nullable: Boolean = false, recordName: String = "topLevelRecord", - prevNameSpace: String = ""): Schema = { + prevNameSpace: String = "", + outputTimestampType: AvroOutputTimestampType.Value = AvroOutputTimestampType.TIMESTAMP_MICROS +): Schema = { --- End diff -- I believe ```scala outputTimestampType: AvroOutputTimestampType.Value = AvroOutputTimestampType.TIMESTAMP_MICROS) : Schema = { ``` is more correct per https://github.com/databricks/scala-style-guide#spacing-and-indentation --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...
Github user HyukjinKwon commented on a diff in the pull request: https://github.com/apache/spark/pull/21935#discussion_r207088878 --- Diff: external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOptions.scala --- @@ -79,4 +80,16 @@ class AvroOptions( val compression: String = { parameters.get("compression").getOrElse(SQLConf.get.avroCompressionCodec) } + + /** + * The `outputTimestampType` option sets which Avro timestamp type to use when Spark writes + * data to Avro files. Currently supported types are `TIMESTAMP_MICROS` and `TIMESTAMP_MILLIS`. + * TIMESTAMP_MICROS is a logical timestamp type in Avro, which stores number of microseconds + * from the Unix epoch. TIMESTAMP_MILLIS is also logical, but with millisecond precision, + * which means Spark has to truncate the microsecond portion of its timestamp value. + */ + val outputTimestampType: AvroOutputTimestampType.Value = { --- End diff -- Hm, I wouldn't expose this as an option for now - that at least matches to Parquet's. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...
Github user gengliangwang commented on a diff in the pull request: https://github.com/apache/spark/pull/21935#discussion_r207000865 --- Diff: external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala --- @@ -103,31 +108,49 @@ object SchemaConverters { catalystType: DataType, nullable: Boolean = false, recordName: String = "topLevelRecord", - prevNameSpace: String = ""): Schema = { + prevNameSpace: String = "", + outputTimestampType: AvroOutputTimestampType.Value = AvroOutputTimestampType.TIMESTAMP_MICROS +): Schema = { --- End diff -- Not sure if the indent here is correct. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/21935#discussion_r206985120 --- Diff: external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala --- @@ -103,31 +108,48 @@ object SchemaConverters { catalystType: DataType, nullable: Boolean = false, recordName: String = "topLevelRecord", - prevNameSpace: String = ""): Schema = { + prevNameSpace: String = "", + outputTimestampType: String = "TIMESTAMP_MICROS"): Schema = { val builder = if (nullable) { SchemaBuilder.builder().nullable() } else { SchemaBuilder.builder() } + catalystType match { case BooleanType => builder.booleanType() case ByteType | ShortType | IntegerType => builder.intType() case LongType => builder.longType() case DateType => builder.longType() - case TimestampType => builder.longType() + case TimestampType => +val timestampType = outputTimestampType match { + case "TIMESTAMP_MILLIS" => LogicalTypes.timestampMillis() --- End diff -- don't hardcode the strings, we can write ``` if (outputTimestampType == AvroOutputTimestampType.TIMESTAMP_MICROS.toString) ... ``` --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/21935#discussion_r206983995 --- Diff: external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala --- @@ -42,7 +43,11 @@ object SchemaConverters { case BYTES => SchemaType(BinaryType, nullable = false) case DOUBLE => SchemaType(DoubleType, nullable = false) case FLOAT => SchemaType(FloatType, nullable = false) - case LONG => SchemaType(LongType, nullable = false) + case LONG => avroSchema.getLogicalType match { +case _: TimestampMillis | _: TimestampMicros => + return SchemaType(TimestampType, nullable = false) --- End diff -- why use `return` here? --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/21935#discussion_r206983421 --- Diff: external/avro/src/main/scala/org/apache/spark/sql/avro/AvroSerializer.scala --- @@ -93,7 +94,11 @@ class AvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: case DateType => (getter, ordinal) => getter.getInt(ordinal) * DateTimeUtils.MILLIS_PER_DAY case TimestampType => -(getter, ordinal) => getter.getLong(ordinal) / 1000 +(getter, ordinal) => avroType.getLogicalType match { + case _: TimestampMillis => getter.getLong(ordinal) / 1000 + case _: TimestampMicros => getter.getLong(ordinal) + case _ => getter.getLong(ordinal) --- End diff -- ditto --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/21935#discussion_r206983054 --- Diff: external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala --- @@ -86,8 +87,14 @@ class AvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType) { case (LONG, LongType) => (updater, ordinal, value) => updater.setLong(ordinal, value.asInstanceOf[Long]) - case (LONG, TimestampType) => (updater, ordinal, value) => -updater.setLong(ordinal, value.asInstanceOf[Long] * 1000) + case (LONG, TimestampType) => avroType.getLogicalType match { +case _: TimestampMillis => (updater, ordinal, value) => + updater.setLong(ordinal, value.asInstanceOf[Long] * 1000) +case _: TimestampMicros => (updater, ordinal, value) => + updater.setLong(ordinal, value.asInstanceOf[Long]) +case _ => (updater, ordinal, value) => + updater.setLong(ordinal, value.asInstanceOf[Long] * 1000) --- End diff -- Let's add a comment to say it's for backward compatibility reasons. Also we should only do it when logical type is null. For other logical types, we should fail here. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/21935#discussion_r206748626 --- Diff: external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala --- @@ -114,7 +121,10 @@ object SchemaConverters { case ByteType | ShortType | IntegerType => builder.intType() case LongType => builder.longType() case DateType => builder.longType() - case TimestampType => builder.longType() + case TimestampType => +// To be consistent with the previous behavior of writing Timestamp type with Avro 1.7, --- End diff -- For now I think writing out timestamp micros should be good --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...
Github user HyukjinKwon commented on a diff in the pull request: https://github.com/apache/spark/pull/21935#discussion_r206747402 --- Diff: external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala --- @@ -35,6 +36,12 @@ object SchemaConverters { * This function takes an avro schema and returns a sql schema. */ def toSqlType(avroSchema: Schema): SchemaType = { +avroSchema.getLogicalType match { --- End diff -- ditto --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...
Github user HyukjinKwon commented on a diff in the pull request: https://github.com/apache/spark/pull/21935#discussion_r206747243 --- Diff: external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala --- @@ -71,7 +72,15 @@ class AvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType) { private def newWriter( avroType: Schema, catalystType: DataType, - path: List[String]): (CatalystDataUpdater, Int, Any) => Unit = + path: List[String]): (CatalystDataUpdater, Int, Any) => Unit = { +(avroType.getLogicalType, catalystType) match { --- End diff -- Can we do this like: ```scala case (LONG, TimestampType) => avroType.getLogicalType match { case _: TimestampMillis => (updater, ordinal, value) => updater.setLong(ordinal, value.asInstanceOf[Long] * 1000) case _: TimestampMicros => (updater, ordinal, value) => updater.setLong(ordinal, value.asInstanceOf[Long]) case _ => (updater, ordinal, value) => updater.setLong(ordinal, value.asInstanceOf[Long] * 1000) } ``` ? Looks they have Avro long type anyway. Thought it's better to read and actually safer and correct. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...
Github user gengliangwang commented on a diff in the pull request: https://github.com/apache/spark/pull/21935#discussion_r206743304 --- Diff: external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala --- @@ -114,7 +121,10 @@ object SchemaConverters { case ByteType | ShortType | IntegerType => builder.intType() case LongType => builder.longType() case DateType => builder.longType() - case TimestampType => builder.longType() + case TimestampType => +// To be consistent with the previous behavior of writing Timestamp type with Avro 1.7, --- End diff -- Previously we write timestamp as `Long` and divide the value by 1000(millisecond precision). Maybe I need to revise the comment. +1 on the new config. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/21935#discussion_r206742399 --- Diff: external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala --- @@ -114,7 +121,10 @@ object SchemaConverters { case ByteType | ShortType | IntegerType => builder.intType() case LongType => builder.longType() case DateType => builder.longType() - case TimestampType => builder.longType() + case TimestampType => +// To be consistent with the previous behavior of writing Timestamp type with Avro 1.7, --- End diff -- also we should follow parquet and have a config `spark.sql.avro.outputTimestampType` to control it. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/21935#discussion_r206742249 --- Diff: external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala --- @@ -114,7 +121,10 @@ object SchemaConverters { case ByteType | ShortType | IntegerType => builder.intType() case LongType => builder.longType() case DateType => builder.longType() - case TimestampType => builder.longType() + case TimestampType => +// To be consistent with the previous behavior of writing Timestamp type with Avro 1.7, +// the default output Avro Timestamp type is with millisecond precision. +builder.longBuilder().prop(LogicalType.LOGICAL_TYPE_PROP, "timestamp-millis").endLong() --- End diff -- is there a better API for it? hardcoding a string is hacky. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/21935#discussion_r206742158 --- Diff: external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala --- @@ -114,7 +121,10 @@ object SchemaConverters { case ByteType | ShortType | IntegerType => builder.intType() case LongType => builder.longType() case DateType => builder.longType() - case TimestampType => builder.longType() + case TimestampType => +// To be consistent with the previous behavior of writing Timestamp type with Avro 1.7, --- End diff -- the previous behavior is: we can't write out timestamp data, isn't it? --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...
GitHub user gengliangwang opened a pull request: https://github.com/apache/spark/pull/21935 [SPARK-24773] Avro: support logical timestamp type with different precisions ## What changes were proposed in this pull request? Support reading Avro logical timestamp type with different precisions https://avro.apache.org/docs/1.8.2/spec.html#Timestamp+%28millisecond+precision%29 Also support writing Spark Timestamp type to Avro timestamp type with millisecond precision, which is consistent with the previous behavior of writing Timestamp type with Avro 1.7. ## How was this patch tested? Unit test You can merge this pull request into a Git repository by running: $ git pull https://github.com/gengliangwang/spark avro_timestamp Alternatively you can review and apply these changes as the patch at: https://github.com/apache/spark/pull/21935.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #21935 commit 3a53f5569fa47cf0fb0e3eadb0f1c134dc1448b0 Author: Gengliang Wang Date: 2018-07-31T18:27:18Z Avro: support logical timestamp type --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org