This is an automated email from the ASF dual-hosted git repository.

dongjoon pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new aca6db1  [SPARK-34434][SQL] Mention DS rebase options in 
`SparkUpgradeException`
aca6db1 is described below

commit aca6db1868c7fd7632b490e5f20254989b270e8e
Author: Max Gekk <[email protected]>
AuthorDate: Sun Feb 14 17:42:15 2021 -0800

    [SPARK-34434][SQL] Mention DS rebase options in `SparkUpgradeException`
    
    ### What changes were proposed in this pull request?
    Mention the DS options introduced by 
https://github.com/apache/spark/pull/31529 and by 
https://github.com/apache/spark/pull/31489 in `SparkUpgradeException`.
    
    ### Why are the changes needed?
    To improve user experience with Spark SQL. Before the changes, the error 
message recommends to set SQL configs but the configs cannot help in the some 
situations (see the PRs for more details).
    
    ### Does this PR introduce _any_ user-facing change?
    Yes. After the changes, the error message is:
    
    _org.apache.spark.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.legacy.parquet.datetimeRebase [...]
    
    ### How was this patch tested?
    1. By checking coding style: `./dev/scalastyle`
    2. By running the related test suite:
    ```
    $ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly 
*ParquetRebaseDatetimeV1Suite"
    ```
    
    Closes #31562 from MaxGekk/rebase-upgrade-exception.
    
    Authored-by: Max Gekk <[email protected]>
    Signed-off-by: Dongjoon Hyun <[email protected]>
---
 .../sql/execution/datasources/DataSourceUtils.scala   | 19 ++++++++++++-------
 1 file changed, 12 insertions(+), 7 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceUtils.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceUtils.scala
index b54747a..a17c317 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceUtils.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceUtils.scala
@@ -30,6 +30,7 @@ import org.apache.spark.sql.{SPARK_LEGACY_DATETIME, 
SPARK_LEGACY_INT96, SPARK_VE
 import org.apache.spark.sql.AnalysisException
 import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogUtils}
 import org.apache.spark.sql.catalyst.util.RebaseDateTime
+import org.apache.spark.sql.execution.datasources.parquet.ParquetOptions
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy
 import org.apache.spark.sql.sources.BaseRelation
@@ -132,19 +133,23 @@ object DataSourceUtils {
   }
 
   def newRebaseExceptionInRead(format: String): SparkUpgradeException = {
-    val config = format match {
-      case "Parquet INT96" => 
SQLConf.LEGACY_PARQUET_INT96_REBASE_MODE_IN_READ.key
-      case "Parquet" => SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_READ.key
-      case "Avro" => SQLConf.LEGACY_AVRO_REBASE_MODE_IN_READ.key
+    val (config, option) = format match {
+      case "Parquet INT96" =>
+        (SQLConf.LEGACY_PARQUET_INT96_REBASE_MODE_IN_READ.key, 
ParquetOptions.INT96_REBASE_MODE)
+      case "Parquet" =>
+        (SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_READ.key, 
ParquetOptions.DATETIME_REBASE_MODE)
+      case "Avro" =>
+        (SQLConf.LEGACY_AVRO_REBASE_MODE_IN_READ.key, "datetimeRebaseMode")
       case _ => throw new IllegalStateException("unrecognized format " + 
format)
     }
     new SparkUpgradeException("3.0", "reading dates before 1582-10-15 or 
timestamps before " +
       s"1900-01-01T00:00:00Z from $format 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 " +
-      s"SPARK-31404. You can set $config to 'LEGACY' to rebase the datetime 
values w.r.t. " +
-      s"the calendar difference during reading. Or set $config to 'CORRECTED' 
to read the " +
-      "datetime values as it is.", null)
+      s"SPARK-31404. You can set the SQL config '$config' or the datasource 
option '$option' to " +
+      "'LEGACY' to rebase the datetime values w.r.t. the calendar difference 
during reading. " +
+      s"To read the datetime values as it is, set the SQL config '$config' or 
" +
+      s"the datasource option '$option' to 'CORRECTED'.", null)
   }
 
   def newRebaseExceptionInWrite(format: String): SparkUpgradeException = {


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to