[ https://issues.apache.org/jira/browse/SPARK-33571?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17241379#comment-17241379 ]
Maxim Gekk commented on SPARK-33571: ------------------------------------ I have tried to reproduce the issue on the master branch by reading the file saved by Spark 2.4.5 (https://github.com/apache/spark/tree/master/sql/core/src/test/resources/test-data): {code:scala} test("SPARK-33571: read ancient dates saved by Spark 2.4.5") { withSQLConf(SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_READ.key -> LEGACY.toString) { val path = getResourceParquetFilePath("test-data/before_1582_date_v2_4_5.snappy.parquet") val df = spark.read.parquet(path) df.show(false) } withSQLConf(SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_READ.key -> CORRECTED.toString) { val path = getResourceParquetFilePath("test-data/before_1582_date_v2_4_5.snappy.parquet") val df = spark.read.parquet(path) df.show(false) } } {code} The results are different in LEGACY and in CORRECTED modes: {code} +----------+----------+ |dict |plain | +----------+----------+ |1001-01-01|1001-01-01| |1001-01-01|1001-01-02| |1001-01-01|1001-01-03| |1001-01-01|1001-01-04| |1001-01-01|1001-01-05| |1001-01-01|1001-01-06| |1001-01-01|1001-01-07| |1001-01-01|1001-01-08| +----------+----------+ +----------+----------+ |dict |plain | +----------+----------+ |1001-01-07|1001-01-07| |1001-01-07|1001-01-08| |1001-01-07|1001-01-09| |1001-01-07|1001-01-10| |1001-01-07|1001-01-11| |1001-01-07|1001-01-12| |1001-01-07|1001-01-13| |1001-01-07|1001-01-14| +----------+----------+ {code} > Handling of hybrid to proleptic calendar when reading and writing Parquet > data not working correctly > ---------------------------------------------------------------------------------------------------- > > Key: SPARK-33571 > URL: https://issues.apache.org/jira/browse/SPARK-33571 > Project: Spark > Issue Type: Bug > Components: PySpark, Spark Core > Affects Versions: 3.0.0, 3.0.1 > Reporter: Simon > Priority: Major > > The handling of old dates written with older Spark versions (<2.4.6) using > the hybrid calendar in Spark 3.0.0 and 3.0.1 seems to be broken/not working > correctly. > From what I understand it should work like this: > * Only relevant for `DateType` before 1582-10-15 or `TimestampType` before > 1900-01-01T00:00:00Z > * Only applies when reading or writing parquet files > * When reading parquet files written with Spark < 2.4.6 which contain dates > or timestamps before the above mentioned moments in time a > `SparkUpgradeException` should be raised informing the user to choose either > `LEGACY` or `CORRECTED` for the `datetimeRebaseModeInRead` > * When reading parquet files written with Spark < 2.4.6 which contain dates > or timestamps before the above mentioned moments in time and > `datetimeRebaseModeInRead` is set to `LEGACY` the dates and timestamps should > show the same values in Spark 3.0.1. with for example `df.show()` as they did > in Spark 2.4.5 > * When reading parquet files written with Spark < 2.4.6 which contain dates > or timestamps before the above mentioned moments in time and > `datetimeRebaseModeInRead` is set to `CORRECTED` the dates and timestamps > should show different values in Spark 3.0.1. with for example `df.show()` as > they did in Spark 2.4.5 > * When writing parqet files with Spark > 3.0.0 which contain dates or > timestamps before the above mentioned moment in time a > `SparkUpgradeException` should be raised informing the user to choose either > `LEGACY` or `CORRECTED` for the `datetimeRebaseModeInWrite` > First of all I'm not 100% sure all of this is correct. I've been unable to > find any clear documentation on the expected behavior. The understanding I > have was pieced together from the mailing list > ([http://apache-spark-user-list.1001560.n3.nabble.com/Spark-3-0-1-new-Proleptic-Gregorian-calendar-td38914.html)] > the blog post linked there and looking at the Spark code. > From our testing we're seeing several issues: > * Reading parquet data with Spark 3.0.1 that was written with Spark 2.4.5. > that contains fields of type `TimestampType` which contain timestamps before > the above mentioned moments in time without `datetimeRebaseModeInRead` set > doesn't raise the `SparkUpgradeException`, it succeeds without any changes to > the resulting dataframe compares to that dataframe in Spark 2.4.5 > * Reading parquet data with Spark 3.0.1 that was written with Spark 2.4.5. > that contains fields of type `TimestampType` or `DateType` which contain > dates or timestamps before the above mentioned moments in time with > `datetimeRebaseModeInRead` set to `LEGACY` results in the same values in the > dataframe as when using `CORRECTED`, so it seems like no rebasing is > happening. > I've made some scripts to help with testing/show the behavior, it uses > pyspark 2.4.5, 2.4.6 and 3.0.1. You can find them here > [https://github.com/simonvanderveldt/spark3-rebasemode-issue]. I'll post the > outputs in a comment below as well. -- This message was sent by Atlassian Jira (v8.3.4#803005) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org