Skip to content
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

[BUG] Job failed with SparkUpgradeException no matter which value are set for spark.sql.parquet.datetimeRebaseModeInRead #9540

Closed
viadea opened this issue Oct 25, 2023 · 10 comments
Assignees
Labels
bug Something isn't working

Comments

@viadea
Copy link
Collaborator

viadea commented Oct 25, 2023

Describe the bug
Below query failed with

Caused by: SparkUpgradeException: You may get a different result due to the upgrading of Spark 3.0: 
reading dates before 1582-10-15 or timestamps before 1900-01-01T00:00:00Z from Parquet
files can be ambiguous, as the files may be written by Spark 2.x or legacy versions of
Hive, which uses a legacy hybrid calendar that is different from Spark 3.0+'s Proleptic
Gregorian calendar. See more details in SPARK-31404. You can set the SQL config
'spark.sql.parquet.datetimeRebaseModeInRead' or the datasource option 'datetimeRebaseMode' to 'LEGACY' to rebase the datetime values
w.r.t. the calendar difference during reading. To read the datetime values as it is,
set the SQL config 'spark.sql.parquet.datetimeRebaseModeInRead' or the datasource option 'datetimeRebaseMode' to 'CORRECTED'.

No matter which value we set below, it always failed with the same error:

spark.conf.set("spark.sql.parquet.datetimeRebaseModeInRead","LEGACY")
spark.conf.set("spark.sql.parquet.datetimeRebaseModeInRead","EXCEPTION")
spark.conf.set("spark.sql.parquet.datetimeRebaseModeInRead","CORRECTED")
spark.conf.set("spark.sql.legacy.parquet.datetimeRebaseModeInRead", "LEGACY")
spark.conf.set("spark.sql.legacy.parquet.datetimeRebaseModeInRead", "EXCEPTION")
spark.conf.set("spark.sql.legacy.parquet.datetimeRebaseModeInRead", "CORRECTED")

CPU run always works fine.

Steps/Code to reproduce bug

spark.conf.set("spark.sql.parquet.datetimeRebaseModeInWrite","LEGACY")

import java.sql.Date
Seq(Date.valueOf("1500-12-25")).toDF("dt").write.format("parquet").mode("overwrite").save("/tmp/testparquet_legacy")
spark.read.parquet("/tmp/testparquet_legacy").createOrReplaceTempView("date_legacy")
spark.sql("SELECT * FROM date_legacy").explain
spark.sql("SELECT * FROM date_legacy").show

Expected behavior
A clear and concise description of what you expected to happen.

Environment details (please complete the following information)
Databricks 10.4 ML LTS + Spark RAPIDS 23.08.2 or 23.06
Databricks 11.3 ML LTS + Spark RAPIDS 23.08.2
On-Prem Apache Spark 3.3 + Spark RAPIDS 23.08.2

@viadea viadea added bug Something isn't working ? - Needs Triage Need team to review and classify labels Oct 25, 2023
@tgravescs
Copy link
Collaborator

its likely set in the parquet file itself and that overrides any of the configs: This is dup of #9059

@viadea
Copy link
Collaborator Author

viadea commented Oct 25, 2023

its likely set in the parquet file itself and that overrides any of the configs: This is dup of #9059

I also tried below but still failed each time:

spark.conf.set("spark.sql.legacy.parquet.datetimeRebaseModeInRead", "LEGACY")
spark.conf.set("spark.sql.parquet.datetimeRebaseModeInRead","LEGACY")

spark.conf.set("spark.sql.legacy.parquet.datetimeRebaseModeInWrite", "LEGACY")
spark.conf.set("spark.sql.parquet.datetimeRebaseModeInWrite","LEGACY")

import java.sql.Date
Seq(Date.valueOf("1500-12-25")).toDF("dt").write.format("parquet").mode("overwrite").save("/tmp/testparquet_legacy")
spark.read.parquet("/tmp/testparquet_legacy").createOrReplaceTempView("date_legacy")
spark.sql("SELECT * FROM date_legacy").explain
spark.sql("SELECT * FROM date_legacy").show

@viadea
Copy link
Collaborator Author

viadea commented Oct 25, 2023

The issue is there is no any workaround.

@viadea
Copy link
Collaborator Author

viadea commented Oct 25, 2023

Also reproduced on local env spark-3.3.2-bin-hadoop3 + spark rapids 23.08.1

@viadea
Copy link
Collaborator Author

viadea commented Oct 25, 2023

Update:
I can make it work in below combination:

spark.conf.set("spark.sql.parquet.datetimeRebaseModeInRead","LEGACY")
spark.conf.set("spark.sql.parquet.datetimeRebaseModeInWrite","CORRECTED")

@ttnghia
Copy link
Collaborator

ttnghia commented Oct 25, 2023

Yes, because datetimeRebaseModeInRead is not supported in LEGACY mode. Tom pointed out above: #9059

@ttnghia
Copy link
Collaborator

ttnghia commented Oct 25, 2023

So currently we can't support a round trip write/read ion LEGACY mode. But we will be able to do so after #9059 is done, which is under active development.

@viadea
Copy link
Collaborator Author

viadea commented Oct 25, 2023

I tested all 9 combinations using Apache Spark 3.3 + spark rapids 23.08.1 between CPU vs GPU mode:

spark.sql.parquet.datetimeRebaseModeInWrite spark.sql.parquet.datetimeRebaseModeInRead CPU Write CPU Read GPU Write GPU Read Note
LEGACY LEGACY Good Good Fallback FAIL  
LEGACY EXCEPTION Good Good Fallback FAIL  
LEGACY CORRECTED Good Good Fallback FAIL  
EXCEPTION LEGACY FAIL   FAIL   Excepted
EXCEPTION EXCEPTION FAIL   FAIL   Excepted
EXCEPTION CORRECTED FAIL   FAIL   Excepted
CORRECTED LEGACY Good Good Good Good  
CORRECTED EXCEPTION Good Good Good Good  
CORRECTED CORRECTED Good Good Good Good

23.10 snapshot jar test:

spark.sql.parquet.datetimeRebaseModeInWrite spark.sql.parquet.datetimeRebaseModeInRead CPU Write CPU Read GPU Write GPU Read Note
LEGACY LEGACY Good Good Good FAIL  
LEGACY EXCEPTION Good Good Good FAIL  
LEGACY CORRECTED Good Good Good FAIL  
EXCEPTION LEGACY FAIL   FAIL   Excepted
EXCEPTION EXCEPTION FAIL   FAIL   Excepted
EXCEPTION CORRECTED FAIL   FAIL   Excepted
CORRECTED LEGACY Good Good Good Good  
CORRECTED EXCEPTION Good Good Good Good  
CORRECTED CORRECTED Good Good Good Good

@viadea
Copy link
Collaborator Author

viadea commented Oct 25, 2023

In summary, we need to make sure the:

spark.sql.parquet.datetimeRebaseModeInWrite=CORRECTED

and no matter what value we set for spark.sql.parquet.datetimeRebaseModeInRead, the write and read can both run on GPU.

@viadea viadea changed the title [BUG] On Databricks the job failed with SparkUpgradeException no matter which value are set for spark.sql.parquet.datetimeRebaseModeInRead [BUG] Job failed with SparkUpgradeException no matter which value are set for spark.sql.parquet.datetimeRebaseModeInRead Oct 25, 2023
@mattahrens
Copy link
Collaborator

Closing as dup of #9059

@mattahrens mattahrens closed this as not planned Won't fix, can't repro, duplicate, stale Oct 31, 2023
@mattahrens mattahrens removed the ? - Needs Triage Need team to review and classify label Oct 31, 2023
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug Something isn't working
Projects
None yet
Development

No branches or pull requests

4 participants