-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-24773] Avro: support logical timestamp type with different precisions #21935
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 6 commits
3a53f55
fdc6c2c
be0077a
09ad6e9
2b286cd
921e6cb
499fbf3
fed8505
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -23,6 +23,7 @@ import scala.collection.JavaConverters._ | |
| import scala.collection.mutable.ArrayBuffer | ||
|
|
||
| import org.apache.avro.{Schema, SchemaBuilder} | ||
| import org.apache.avro.LogicalTypes.{TimestampMicros, TimestampMillis} | ||
| import org.apache.avro.Schema.Type._ | ||
| import org.apache.avro.generic._ | ||
| import org.apache.avro.util.Utf8 | ||
|
|
@@ -86,8 +87,18 @@ 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) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 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. |
||
| case other => throw new IncompatibleSchemaException( | ||
| s"Cannot convert Avro logical type ${other} to Catalyst Timestamp type.") | ||
| } | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. we should add a default case and throw IncompatibleSchemaException, in case avro add more logical types for long type in the future. |
||
|
|
||
| case (LONG, DateType) => (updater, ordinal, value) => | ||
| updater.setInt(ordinal, (value.asInstanceOf[Long] / DateTimeUtils.MILLIS_PER_DAY).toInt) | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -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") | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. at least we should provide how the binary file is generated, or just do roundtrip test: Spark write avro files and then read it.
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The schema and data is stated in https://github.com/apache/spark/pull/21935/files#diff-9364b0610f92b3cc35a4bc43a80751bfR397 |
||
|
|
||
| override protected def beforeAll(): Unit = { | ||
| super.beforeAll() | ||
|
|
@@ -331,6 +332,84 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { | |
| } | ||
| } | ||
|
|
||
| test("Logical type: timestamp_millis") { | ||
| val sparkSession = spark | ||
| import sparkSession.implicits._ | ||
|
|
||
| val expected = | ||
| Seq(1L, 666L).toDF("timestamp_millis").select('timestamp_millis.cast(TimestampType)).collect() | ||
| val df = spark.read.format("avro").load(timestampAvro).select('timestamp_millis) | ||
|
|
||
| checkAnswer(df, expected) | ||
|
|
||
| withTempPath { dir => | ||
| df.write.format("avro").save(dir.toString) | ||
| checkAnswer(spark.read.format("avro").load(dir.toString), expected) | ||
| } | ||
| } | ||
|
|
||
| test("Logical type: timestamp_micros") { | ||
| val sparkSession = spark | ||
| import sparkSession.implicits._ | ||
|
|
||
| val expected = | ||
| Seq(2L, 999L).toDF("timestamp_micros").select('timestamp_micros.cast(TimestampType)).collect() | ||
| val df = spark.read.format("avro").load(timestampAvro).select('timestamp_micros) | ||
|
|
||
| checkAnswer(df, expected) | ||
|
|
||
| withTempPath { dir => | ||
| df.write.format("avro").save(dir.toString) | ||
| checkAnswer(spark.read.format("avro").load(dir.toString), expected) | ||
| } | ||
| } | ||
|
|
||
| test("Logical type: specify different output timestamp types") { | ||
| val sparkSession = spark | ||
| import sparkSession.implicits._ | ||
|
|
||
| val df = spark.read.format("avro").load(timestampAvro) | ||
|
|
||
| val expected = Seq((1L, 2L), (666L, 999L)) | ||
| .toDF("timestamp_millis", "timestamp_micros") | ||
| .select('timestamp_millis.cast(TimestampType), 'timestamp_micros.cast(TimestampType)) | ||
| .collect() | ||
|
|
||
| Seq("TIMESTAMP_MILLIS", "TIMESTAMP_MICROS").foreach { timestampType => | ||
| withSQLConf(SQLConf.AVRO_OUTPUT_TIMESTAMP_TYPE.key -> timestampType) { | ||
| withTempPath { dir => | ||
| df.write.format("avro").save(dir.toString) | ||
| checkAnswer(spark.read.format("avro").load(dir.toString), expected) | ||
| } | ||
| } | ||
| } | ||
| } | ||
|
|
||
| test("Logical type: user specified schema") { | ||
| val sparkSession = spark | ||
| import sparkSession.implicits._ | ||
|
|
||
| val expected = Seq((1L, 2L), (666L, 999L)) | ||
| .toDF("timestamp_millis", "timestamp_micros") | ||
| .select('timestamp_millis.cast(TimestampType), 'timestamp_micros.cast(TimestampType)) | ||
| .collect() | ||
|
|
||
| val avroSchema = s""" | ||
| { | ||
| "namespace": "logical", | ||
| "type": "record", | ||
| "name": "test", | ||
| "fields": [ | ||
| {"name": "timestamp_millis", "type": {"type": "long","logicalType": "timestamp-millis"}}, | ||
| {"name": "timestamp_micros", "type": {"type": "long","logicalType": "timestamp-micros"}} | ||
| ] | ||
| } | ||
| """ | ||
| val df = spark.read.format("avro").option("avroSchema", avroSchema).load(timestampAvro) | ||
|
|
||
| checkAnswer(df, expected) | ||
| } | ||
|
|
||
| test("Array data types") { | ||
| withTempPath { dir => | ||
| val testSchema = StructType(Seq( | ||
|
|
@@ -511,7 +590,8 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { | |
|
|
||
| // TimesStamps are converted to longs | ||
| val times = spark.read.format("avro").load(avroDir).select("Time").collect() | ||
| assert(times.map(_(0)).toSet == Set(666, 777, 42)) | ||
| assert(times.map(_(0)).toSet == | ||
| Set(new Timestamp(666), new Timestamp(777), new Timestamp(42))) | ||
|
|
||
| // DecimalType should be converted to string | ||
| val decimals = spark.read.format("avro").load(avroDir).select("Decimal").collect() | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
ditto, add a default case.