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 2022/02/28 09:40:01 UTC

[GitHub] [spark] MaxGekk commented on a change in pull request #35670: [SPARK-38112][SQL] Use error classes in the execution errors of date/timestamp handling

MaxGekk commented on a change in pull request #35670:
URL: https://github.com/apache/spark/pull/35670#discussion_r815717311



##########
File path: core/src/main/scala/org/apache/spark/SparkException.scala
##########
@@ -71,9 +71,35 @@ private[spark] case class ExecutorDeadException(message: String)
 /**
  * Exception thrown when Spark returns different result after upgrading to a new version.
  */
-private[spark] class SparkUpgradeException(version: String, message: String, cause: Throwable)
-  extends RuntimeException("You may get a different result due to the upgrading of Spark" +
-    s" $version: $message", cause)
+private[spark] class SparkUpgradeException(
+    version: String,
+    message: String,
+    cause: Throwable,
+    errorClass: Option[String],
+    messageParameters: Array[String])
+  extends RuntimeException(message, cause) with SparkThrowable {
+
+  def this(version: String, message: String, cause: Throwable) =
+    this (
+      version = version,
+      message = s"You may get a different result due to the upgrading of Spark $version: $message",

Review comment:
       Let's avoid error message duplication. error-classes.json should be one source of truth:
   ```
     "INCONSISTENT_BEHAVIOR_CROSS_VERSION" : {
       "message" : [ "You may get a different result due to the upgrading of Spark %s: %s" ]
     },
   ```

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala
##########
@@ -171,4 +181,72 @@ class QueryExecutionErrorsSuite extends QueryTest with SharedSparkSession {
     assert(e2.getSqlState === "0A000")
     assert(e2.getMessage === "The feature is not supported: Pivot not after a groupBy.")
   }
+
+  test("INCONSISTENT_BEHAVIOR_CROSS_VERSION: " +
+    "compatibility with Spark 2.4/3.2 in reading/writing dates") {
+
+    // Fail to read ancient datetime values.
+    withSQLConf(SQLConf.PARQUET_REBASE_MODE_IN_READ.key -> EXCEPTION.toString) {
+      val fileName = "before_1582_date_v2_4_5.snappy.parquet"
+      val filePath = getResourceParquetFilePath("test-data/" + fileName)
+      val e = intercept[SparkException] {
+        spark.read.parquet(filePath).collect()
+      }.getCause.asInstanceOf[SparkUpgradeException]
+
+      assert(e.getErrorClass === "INCONSISTENT_BEHAVIOR_CROSS_VERSION")
+      assert(e.getMessage
+        .startsWith("You may get a different result due to the upgrading of Spark 3.0: \n" +

Review comment:
       The message confuses slightly. We run Spark 3.3.0-SNAPSHOT (almost 3.3) and try to read a file written by Spark 2.4 but the message says: due to the **upgrading of Spark 3.0**.
   
   Should be somehow: ... due to upgrading to Spark >= 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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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