You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2020/03/18 18:36:37 UTC

[GitHub] [spark] MaxGekk opened a new pull request #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro

MaxGekk opened a new pull request #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro
URL: https://github.com/apache/spark/pull/27953
 
 
   ### What changes were proposed in this pull request?
   The PR addresses the issue of compatibility with Spark 2.4 and earlier version in reading/writing dates and timestamp via Avro datasource. Previous releases are based on a hybrid calendar - Julian + Gregorian. Since Spark 3.0, Proleptic Gregorian calendar is used by default, see SPARK-26651. In particular, the issue pops up for dates/timestamps before 1582-10-15 when the hybrid calendar switches from/to Gregorian to/from Julian calendar. The same local date in different calendar is converted to different number of days since the epoch 1970-01-01. For example, the 1001-01-01 date is converted to:
   - -719164 in Julian calendar. Spark 2.4 saves the number as a value of DATE type into avro files.
   - -719162 in Proleptic Gregorian calendar. Spark 3.0 saves the number as a date value.
   
   The PR proposes rebasing from/to Proleptic Gregorian calendar to the hybrid one under the SQL config:
   ```
   spark.sql.legacy.avro.rebaseDateTime.enabled
   ```
   which is set to `false` by default which means the rebasing is not performed by default.
   
   The details of the implementation:
   1. Added 2 methods to `DateTimeUtils` for rebasing microseconds. `rebaseGregorianToJulianMicros()` builds a local timestamp in Proleptic Gregorian calendar, extracts date-time fields `year`, `month`, ..., `second fraction` from the local timestamp and uses them to build another local timestamp based on the hybrid calendar (using `java.util.Calendar` API). After that it calculates the number of microseconds since the epoch using the resulted local timestamp. The function performs the conversion via the system JVM time zone for compatibility with Spark 2.4 and earlier versions. The `rebaseJulianToGregorianMicros()` function does reverse conversion.
   2. Added 2 methods to `DateTimeUtils` for rebasing days. `rebaseGregorianToJulianDays()` builds a local date from the passed number of days since the epoch in Proleptic Gregorian calendar, interprets the resulted date as a local date in the hybrid calendar and gets the number of days since the epoch from the resulted local date. The conversion is performed via the `UTC` time zone because the conversion is independent from time zones, and `UTC` is selected to void round issues of casting days to milliseconds and back. The `rebaseJulianToGregorianDays()` functions does revers conversion.
   3. Use `rebaseGregorianToJulianMicros()` and `rebaseGregorianToJulianDays()` while saving timestamps/dates to avro files if the SQL config is on.
   4. Use `rebaseJulianToGregorianMicros()` and `rebaseJulianToGregorianDays()` while loading timestamps/dates from avro files if the SQL config is on.
   5. The SQL config `spark.sql.legacy.avro.rebaseDateTime.enabled` controls conversions from/to dates, and timestamps of the `timestamp-millis`, `timestamp-micros` logical types, see the SQL config `spark.sql.avro.outputTimestampType`.
   6. The rebasing is always performed for `INT96` timestamps, independently from `spark.sql.legacy.avro.rebaseDateTime.enabled`.
   
   ### Why are the changes needed?
   For the backward compatibility with Spark 2.4 and earlier versions. The changes allow users to read dates/timestamps saved by previous version, and get the same result. Also after the changes, users can enable the rebasing in write, and save dates/timestamps that can be loaded correctly by Spark 2.4 and earlier versions.
   
   ### Does this PR introduce any user-facing change?
   Yes, the timestamp `1001-01-01 01:02:03.123456` saved by Spark 2.4.5 as `TIMESTAMP_MICROS` is interpreted by Spark 3.0.0-preview2 differently:
   ```scala
   scala> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles")
   scala> spark.read.format("avro").load("/Users/maxim/tmp/before_1582/2_4_5_date_avro").show(false)
   +----------+
   |date      |
   +----------+
   |1001-01-07|
   +----------+
   ```
   After the changes:
   ```scala
   scala> spark.conf.set("spark.sql.legacy.avro.rebaseDateTime.enabled", true)
   scala> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles")
   
   scala> spark.read.format("avro").load("/Users/maxim/tmp/before_1582/2_4_5_date_avro").show(false)
   +----------+
   |date      |
   +----------+
   |1001-01-01|
   +----------+
   ```
   
   ### How was this patch tested?
   1. Added tests to `AvroLogicalTypeSuite` to check rebasing in read. The test reads back avro files saved by Spark 2.4.5 via:
   ```shell
   $ export TZ="America/Los_Angeles"
   ```
   ```scala
   scala> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles")
   scala> val df = Seq("1001-01-01").toDF("dateS").select($"dateS".cast("date").as("date"))
   df: org.apache.spark.sql.DataFrame = [date: date]
   scala> spark.read.format("avro").load("/Users/maxim/tmp/before_1582/2_4_5_ts_avro").show(false)
   scala> df2.write.format("avro").save("/Users/maxim/tmp/before_1582/2_4_5_ts_avro")
   ```
   2. Manually check the write code path. Save date/timestamps (`timestamp-millis`, `timestamp-micros`) by Spark 3.1.0-SNAPSHOT (after the changes):
   ```bash
   $ export TZ="America/Los_Angeles"
   ```
   ```scala
   scala> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles")
   scala> spark.conf.set("spark.sql.legacy.avro.rebaseDateTime.enabled", true)
   TODO
   ```
   Read the saved date/timestamp by Spark 2.4.5:
   ```scala
   TODO
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601113971
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] MaxGekk commented on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601117472
 
 
   @cloud-fan @HyukjinKwon Please, review the PR.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-600905484
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120003/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601254844
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120048/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] MaxGekk commented on a change in pull request #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on a change in pull request #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro
URL: https://github.com/apache/spark/pull/27953#discussion_r395467080
 
 

 ##########
 File path: external/avro/src/test/scala/org/apache/spark/sql/avro/AvroLogicalTypeSuite.scala
 ##########
 @@ -348,6 +348,100 @@ abstract class AvroLogicalTypeSuite extends QueryTest with SharedSparkSession {
       assert(msg.contains("Unscaled value too large for precision"))
     }
   }
+
+  private def readResourceAvroFile(name: String): DataFrame = {
+    val url = Thread.currentThread().getContextClassLoader.getResource(name)
+    spark.read.format("avro").load(url.toString)
+  }
+
+  test("SPARK-31183: compatibility with Spark 2.4 in reading dates/timestamps") {
 
 Review comment:
   Do you mean only this test, correct?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601450823
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601221892
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120045/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601450823
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601094438
 
 
   **[Test build #120045 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120045/testReport)** for PR 27953 at commit [`2e1cee1`](https://github.com/apache/spark/commit/2e1cee17d4803ac663530f6dfbc182ff28c4acaf).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] MaxGekk commented on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-600795868
 
 
   cc @cloud-fan 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601450829
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120067/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601221869
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] MaxGekk commented on a change in pull request #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on a change in pull request #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro
URL: https://github.com/apache/spark/pull/27953#discussion_r395225334
 
 

 ##########
 File path: external/avro/src/test/scala/org/apache/spark/sql/avro/AvroLogicalTypeSuite.scala
 ##########
 @@ -348,6 +348,90 @@ abstract class AvroLogicalTypeSuite extends QueryTest with SharedSparkSession {
       assert(msg.contains("Unscaled value too large for precision"))
     }
   }
+
+  private def readResourceAvroFile(name: String): DataFrame = {
+    val url = Thread.currentThread().getContextClassLoader.getResource(name)
+    spark.read.format("avro").load(url.toString)
+  }
+
+  test("SPARK-31183: compatibility with Spark 2.4 in reading dates/timestamps") {
+    withSQLConf(SQLConf.LEGACY_AVRO_REBASE_DATETIME.key -> "true") {
+      checkAnswer(
+        readResourceAvroFile("before_1582_date_v2_4.avro"),
+        Row(java.sql.Date.valueOf("1001-01-01")))
+      checkAnswer(readResourceAvroFile(
+        "before_1582_ts_micros_v2_4.avro"),
+        Row(java.sql.Timestamp.valueOf("1001-01-01 01:02:03.123456")))
+    }
+  }
+
+  test("SPARK-31183: rebasing microseconds timestamps in write") {
+    val tsStr = "1001-01-01 01:02:03.123456"
+    val nonRebased = "1001-01-07 01:09:05.123456"
+    withTempPath { dir =>
+      val path = dir.getAbsolutePath
+      withSQLConf(SQLConf.LEGACY_AVRO_REBASE_DATETIME.key -> "true") {
+        Seq(tsStr).toDF("tsS")
+          .select($"tsS".cast("timestamp").as("ts"))
+          .write.format("avro")
+          .save(path)
+
+        checkAnswer(spark.read.format("avro").load(path), Row(Timestamp.valueOf(tsStr)))
+      }
+      withSQLConf(SQLConf.LEGACY_AVRO_REBASE_DATETIME.key -> "false") {
+        checkAnswer(spark.read.format("avro").load(path), Row(Timestamp.valueOf(nonRebased)))
+      }
+    }
+  }
+
+  test("SPARK-31183: rebasing milliseconds timestamps in write") {
+    val tsStr = "1001-01-01 01:02:03.123456"
+    val rebased = "1001-01-01 01:02:03.123"
+    val nonRebased = "1001-01-07 01:09:05.123"
+    val timestampSchema = """
 
 Review comment:
   done

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan closed pull request #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro

Posted by GitBox <gi...@apache.org>.
cloud-fan closed pull request #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro
URL: https://github.com/apache/spark/pull/27953
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601113374
 
 
   **[Test build #120048 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120048/testReport)** for PR 27953 at commit [`dac03f2`](https://github.com/apache/spark/commit/dac03f291ba2175ffcb6b374ddb669f0a60b7f8e).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601094974
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] MaxGekk commented on a change in pull request #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on a change in pull request #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro
URL: https://github.com/apache/spark/pull/27953#discussion_r395466971
 
 

 ##########
 File path: external/avro/src/main/scala/org/apache/spark/sql/avro/AvroSerializer.scala
 ##########
 @@ -135,15 +140,26 @@ class AvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable:
       case (BinaryType, BYTES) =>
         (getter, ordinal) => ByteBuffer.wrap(getter.getBinary(ordinal))
 
+      case (DateType, INT) if rebaseDateTime =>
+        (getter, ordinal) => DateTimeUtils.rebaseGregorianToJulianDays(getter.getInt(ordinal))
+
       case (DateType, INT) =>
         (getter, ordinal) => getter.getInt(ordinal)
 
       case (TimestampType, LONG) => avroType.getLogicalType match {
-          case _: TimestampMillis => (getter, ordinal) => getter.getLong(ordinal) / 1000
-          case _: TimestampMicros => (getter, ordinal) => 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, ordinal) => getter.getLong(ordinal) / 1000
+          // For backward compatibility, if the Avro type is Long and it is not logical type
+          // (the `null` case), output the timestamp value as with millisecond precision.
+          case null | _: TimestampMillis => (getter, ordinal) =>
+            val micros = getter.getLong(ordinal)
+            val rebasedMicros = if (rebaseDateTime) {
 
 Review comment:
   1. I assumed timestamps in milliseconds is rare case. By default, Spark writes microseconds.
   2. Checking the boolean flag shouldn't have significant overhead.
   3. If the function is hot, jvm should optimize it
   
   I can move the flag checking out of the function body in a follow PR, or in the same for https://github.com/apache/spark/pull/27953#discussion_r395453034

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-600904882
 
 
   **[Test build #120003 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120003/testReport)** for PR 27953 at commit [`9a96af0`](https://github.com/apache/spark/commit/9a96af08260967554a9ae7c4fa9bd8d66a223e85).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601221869
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] MaxGekk edited a comment on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro

Posted by GitBox <gi...@apache.org>.
MaxGekk edited a comment on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601092719
 
 
   @cloud-fan I cannot test the `timestamp-millis` saved by Spark 2.4.5 because it doesn't allow to save timestamps of the type, see:
   ```scala
   scala> :paste
   // Entering paste mode (ctrl-D to finish)
   
     val timestampSchema = s"""
         {
           "namespace": "logical",
           "type": "record",
           "name": "test",
           "fields": [
             {"name": "ts", "type": {"type": "long","logicalType": "timestamp-millis"}}
           ]
         }
       """
   
   // Exiting paste mode, now interpreting.
   
   timestampSchema: String =
   "
         {
           "namespace": "logical",
           "type": "record",
           "name": "test",
           "fields": [
             {"name": "ts", "type": {"type": "long","logicalType": "timestamp-millis"}}
           ]
         }
       "
   
   scala> val df3 = Seq("1001-01-01 01:02:03.123456").toDF("tsS").select($"tsS".cast("timestamp").as("ts"))
   df3: org.apache.spark.sql.DataFrame = [ts: timestamp]
   
   scala> df3.write.format("avro").option("avroSchema", timestampSchema).save("/Users/maxim/tmp/before_1582/2_4_5_ts_millis_avro")
   20/03/19 03:00:37 ERROR Utils: Aborting task
   org.apache.avro.AvroRuntimeException: Not a union: {"type":"long","logicalType":"timestamp-millis"}
   ```
   The same works on the master.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-600797873
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro
URL: https://github.com/apache/spark/pull/27953#discussion_r395471144
 
 

 ##########
 File path: external/avro/src/test/scala/org/apache/spark/sql/avro/AvroLogicalTypeSuite.scala
 ##########
 @@ -348,6 +348,100 @@ abstract class AvroLogicalTypeSuite extends QueryTest with SharedSparkSession {
       assert(msg.contains("Unscaled value too large for precision"))
     }
   }
+
+  private def readResourceAvroFile(name: String): DataFrame = {
+    val url = Thread.currentThread().getContextClassLoader.getResource(name)
+    spark.read.format("avro").load(url.toString)
+  }
+
+  test("SPARK-31183: compatibility with Spark 2.4 in reading dates/timestamps") {
 
 Review comment:
   All the new tests added here. The are more about compatibility, not logical type.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-600797873
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] MaxGekk edited a comment on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro

Posted by GitBox <gi...@apache.org>.
MaxGekk edited a comment on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601335370
 
 
   > It's probably because the actual column is nullable ...
   
   @cloud-fan You are right. I changed the schema while writing by Spark 2.4 and everything is ok.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601341509
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/24784/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-600797887
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/24723/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HyukjinKwon commented on a change in pull request #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on a change in pull request #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro
URL: https://github.com/apache/spark/pull/27953#discussion_r395463351
 
 

 ##########
 File path: external/avro/src/main/scala/org/apache/spark/sql/avro/AvroSerializer.scala
 ##########
 @@ -135,15 +140,26 @@ class AvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable:
       case (BinaryType, BYTES) =>
         (getter, ordinal) => ByteBuffer.wrap(getter.getBinary(ordinal))
 
+      case (DateType, INT) if rebaseDateTime =>
+        (getter, ordinal) => DateTimeUtils.rebaseGregorianToJulianDays(getter.getInt(ordinal))
+
       case (DateType, INT) =>
         (getter, ordinal) => getter.getInt(ordinal)
 
       case (TimestampType, LONG) => avroType.getLogicalType match {
-          case _: TimestampMillis => (getter, ordinal) => getter.getLong(ordinal) / 1000
-          case _: TimestampMicros => (getter, ordinal) => 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, ordinal) => getter.getLong(ordinal) / 1000
+          // For backward compatibility, if the Avro type is Long and it is not logical type
+          // (the `null` case), output the timestamp value as with millisecond precision.
+          case null | _: TimestampMillis => (getter, ordinal) =>
+            val micros = getter.getLong(ordinal)
+            val rebasedMicros = if (rebaseDateTime) {
 
 Review comment:
   One more thing, why don't we return a function rather than checking `rebaseDateTime` for every time?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-600905477
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-600905477
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601254832
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601221892
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120045/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601113971
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601220750
 
 
   **[Test build #120045 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120045/testReport)** for PR 27953 at commit [`2e1cee1`](https://github.com/apache/spark/commit/2e1cee17d4803ac663530f6dfbc182ff28c4acaf).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `sealed trait V1FallbackWriters extends V2CommandExec with SupportsV1Write `
     * `abstract class V2CommandExec extends SparkPlan `
     * `trait V2TableWriteExec extends V2CommandExec with UnaryExecNode `

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601094982
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/24762/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro
URL: https://github.com/apache/spark/pull/27953#discussion_r395013912
 
 

 ##########
 File path: external/avro/src/test/scala/org/apache/spark/sql/avro/AvroLogicalTypeSuite.scala
 ##########
 @@ -348,6 +348,90 @@ abstract class AvroLogicalTypeSuite extends QueryTest with SharedSparkSession {
       assert(msg.contains("Unscaled value too large for precision"))
     }
   }
+
+  private def readResourceAvroFile(name: String): DataFrame = {
+    val url = Thread.currentThread().getContextClassLoader.getResource(name)
+    spark.read.format("avro").load(url.toString)
+  }
+
+  test("SPARK-31183: compatibility with Spark 2.4 in reading dates/timestamps") {
+    withSQLConf(SQLConf.LEGACY_AVRO_REBASE_DATETIME.key -> "true") {
+      checkAnswer(
+        readResourceAvroFile("before_1582_date_v2_4.avro"),
+        Row(java.sql.Date.valueOf("1001-01-01")))
+      checkAnswer(readResourceAvroFile(
+        "before_1582_ts_micros_v2_4.avro"),
+        Row(java.sql.Timestamp.valueOf("1001-01-01 01:02:03.123456")))
+    }
+  }
+
+  test("SPARK-31183: rebasing microseconds timestamps in write") {
+    val tsStr = "1001-01-01 01:02:03.123456"
+    val nonRebased = "1001-01-07 01:09:05.123456"
+    withTempPath { dir =>
+      val path = dir.getAbsolutePath
+      withSQLConf(SQLConf.LEGACY_AVRO_REBASE_DATETIME.key -> "true") {
+        Seq(tsStr).toDF("tsS")
+          .select($"tsS".cast("timestamp").as("ts"))
+          .write.format("avro")
+          .save(path)
+
+        checkAnswer(spark.read.format("avro").load(path), Row(Timestamp.valueOf(tsStr)))
+      }
+      withSQLConf(SQLConf.LEGACY_AVRO_REBASE_DATETIME.key -> "false") {
+        checkAnswer(spark.read.format("avro").load(path), Row(Timestamp.valueOf(nonRebased)))
+      }
+    }
+  }
+
+  test("SPARK-31183: rebasing milliseconds timestamps in write") {
+    val tsStr = "1001-01-01 01:02:03.123456"
+    val rebased = "1001-01-01 01:02:03.123"
+    val nonRebased = "1001-01-07 01:09:05.123"
+    val timestampSchema = """
 
 Review comment:
   nit: can we use the Scala multi line string?
   ```
   """
     |abc
     |xyz
   """.stripMargin
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-600797887
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/24723/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] MaxGekk commented on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601092719
 
 
   @cloud-fan I cannot test the `timestamp_millis` saved by Spark 2.4.5 because it doesn't allow to save timestamps of the type, see:
   ```scala
   scala> :paste
   // Entering paste mode (ctrl-D to finish)
   
     val timestampSchema = s"""
         {
           "namespace": "logical",
           "type": "record",
           "name": "test",
           "fields": [
             {"name": "ts", "type": {"type": "long","logicalType": "timestamp-millis"}}
           ]
         }
       """
   
   // Exiting paste mode, now interpreting.
   
   timestampSchema: String =
   "
         {
           "namespace": "logical",
           "type": "record",
           "name": "test",
           "fields": [
             {"name": "ts", "type": {"type": "long","logicalType": "timestamp-millis"}}
           ]
         }
       "
   
   scala> val df3 = Seq("1001-01-01 01:02:03.123456").toDF("tsS").select($"tsS".cast("timestamp").as("ts"))
   df3: org.apache.spark.sql.DataFrame = [ts: timestamp]
   
   scala> df3.write.format("avro").option("avroSchema", timestampSchema).save("/Users/maxim/tmp/before_1582/2_4_5_ts_millis_avro")
   20/03/19 03:00:37 ERROR Utils: Aborting task
   org.apache.avro.AvroRuntimeException: Not a union: {"type":"long","logicalType":"timestamp-millis"}
   ```
   The same works on the master.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601548659
 
 
   thanks, merging to master/3.0!

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601341509
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/24784/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601113982
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/24765/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601253482
 
 
   **[Test build #120048 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120048/testReport)** for PR 27953 at commit [`dac03f2`](https://github.com/apache/spark/commit/dac03f291ba2175ffcb6b374ddb669f0a60b7f8e).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] MaxGekk commented on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601335370
 
 
   > It's probably because the actual column is nullable ...
   
   You are right. I changed the schema while writing by Spark 2.4 and everything is ok.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601341495
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601113982
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/24765/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-600797100
 
 
   **[Test build #120003 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120003/testReport)** for PR 27953 at commit [`9a96af0`](https://github.com/apache/spark/commit/9a96af08260967554a9ae7c4fa9bd8d66a223e85).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601450236
 
 
   **[Test build #120067 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120067/testReport)** for PR 27953 at commit [`2464c90`](https://github.com/apache/spark/commit/2464c905045856fdbf17d2765ae499b96f912621).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-600905484
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120003/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601254844
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120048/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601094974
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro
URL: https://github.com/apache/spark/pull/27953#discussion_r395453034
 
 

 ##########
 File path: external/avro/src/test/scala/org/apache/spark/sql/avro/AvroLogicalTypeSuite.scala
 ##########
 @@ -348,6 +348,100 @@ abstract class AvroLogicalTypeSuite extends QueryTest with SharedSparkSession {
       assert(msg.contains("Unscaled value too large for precision"))
     }
   }
+
+  private def readResourceAvroFile(name: String): DataFrame = {
+    val url = Thread.currentThread().getContextClassLoader.getResource(name)
+    spark.read.format("avro").load(url.toString)
+  }
+
+  test("SPARK-31183: compatibility with Spark 2.4 in reading dates/timestamps") {
 
 Review comment:
   missed one thing. I think the test is not very related to logical types and probably should be put in `AvroSuite`.
   
   @MaxGekk can you move the test in your next PR?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601094982
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/24762/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601346550
 
 
   **[Test build #120067 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120067/testReport)** for PR 27953 at commit [`2464c90`](https://github.com/apache/spark/commit/2464c905045856fdbf17d2765ae499b96f912621).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] MaxGekk commented on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601109314
 
 
   @gengliangwang Is it possible to save timestamps as `timestamp-millis` by Spark 2.4? see https://github.com/apache/spark/pull/27953#issuecomment-601092719

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HyukjinKwon commented on a change in pull request #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on a change in pull request #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro
URL: https://github.com/apache/spark/pull/27953#discussion_r395477491
 
 

 ##########
 File path: external/avro/src/main/scala/org/apache/spark/sql/avro/AvroSerializer.scala
 ##########
 @@ -135,15 +140,26 @@ class AvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable:
       case (BinaryType, BYTES) =>
         (getter, ordinal) => ByteBuffer.wrap(getter.getBinary(ordinal))
 
+      case (DateType, INT) if rebaseDateTime =>
+        (getter, ordinal) => DateTimeUtils.rebaseGregorianToJulianDays(getter.getInt(ordinal))
+
       case (DateType, INT) =>
         (getter, ordinal) => getter.getInt(ordinal)
 
       case (TimestampType, LONG) => avroType.getLogicalType match {
-          case _: TimestampMillis => (getter, ordinal) => getter.getLong(ordinal) / 1000
-          case _: TimestampMicros => (getter, ordinal) => 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, ordinal) => getter.getLong(ordinal) / 1000
+          // For backward compatibility, if the Avro type is Long and it is not logical type
+          // (the `null` case), output the timestamp value as with millisecond precision.
+          case null | _: TimestampMillis => (getter, ordinal) =>
+            val micros = getter.getLong(ordinal)
+            val rebasedMicros = if (rebaseDateTime) {
 
 Review comment:
   I think it's easy to switch with almost no additional complexity. Seems fine to change rather than relying on other optimization like JIT, or having a bad example.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601346550
 
 
   **[Test build #120067 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120067/testReport)** for PR 27953 at commit [`2464c90`](https://github.com/apache/spark/commit/2464c905045856fdbf17d2765ae499b96f912621).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601170774
 
 
   > Is it possible to save timestamps as timestamp-millis by Spark 2.4?
   
   It's probably because the actual column is nullable (after the cast), but the specified schema is not. Maybe we've fixed something in 3.0.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601094438
 
 
   **[Test build #120045 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120045/testReport)** for PR 27953 at commit [`2e1cee1`](https://github.com/apache/spark/commit/2e1cee17d4803ac663530f6dfbc182ff28c4acaf).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601113374
 
 
   **[Test build #120048 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120048/testReport)** for PR 27953 at commit [`dac03f2`](https://github.com/apache/spark/commit/dac03f291ba2175ffcb6b374ddb669f0a60b7f8e).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27953: [WIP][SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-600797100
 
 
   **[Test build #120003 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120003/testReport)** for PR 27953 at commit [`9a96af0`](https://github.com/apache/spark/commit/9a96af08260967554a9ae7c4fa9bd8d66a223e85).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601254832
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601450829
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120067/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27953: [SPARK-31183][SQL] Rebase date/timestamp from/to Julian calendar in Avro
URL: https://github.com/apache/spark/pull/27953#issuecomment-601341495
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org