You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by ma...@apache.org on 2023/02/08 05:26:23 UTC

[spark] branch branch-3.4 updated: [SPARK-42315][SQL] Assign name to _LEGACY_ERROR_TEMP_(2091|2092)

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

maxgekk pushed a commit to branch branch-3.4
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.4 by this push:
     new a05bc838fcb [SPARK-42315][SQL] Assign name to _LEGACY_ERROR_TEMP_(2091|2092)
a05bc838fcb is described below

commit a05bc838fcbc1bf918016c854dea9e557bd1b5b6
Author: itholic <ha...@databricks.com>
AuthorDate: Wed Feb 8 10:25:49 2023 +0500

    [SPARK-42315][SQL] Assign name to _LEGACY_ERROR_TEMP_(2091|2092)
    
    ### What changes were proposed in this pull request?
    
    This PR proposes to assign name to _LEGACY_ERROR_TEMP_2091 and _LEGACY_ERROR_TEMP_2091, "CANNOT_READ_FOOTER".
    
    ### Why are the changes needed?
    
    We should assign proper name to _LEGACY_ERROR_TEMP_*
    
    ### Does this PR introduce _any_ user-facing change?
    
    No
    
    ### How was this patch tested?
    
    `./build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite*`
    
    Closes #39889 from itholic/LEGACY_2092.
    
    Authored-by: itholic <ha...@databricks.com>
    Signed-off-by: Max Gekk <ma...@gmail.com>
    (cherry picked from commit 3a9c867a59d60f29438d07bd042f1cd037110790)
    Signed-off-by: Max Gekk <ma...@gmail.com>
---
 core/src/main/resources/error/error-classes.json   | 15 +++++---------
 .../spark/sql/errors/QueryExecutionErrors.scala    | 11 ++--------
 .../datasources/parquet/ParquetFileFormat.scala    |  2 +-
 .../execution/datasources/orc/OrcQuerySuite.scala  | 12 +++++++----
 .../execution/datasources/orc/OrcSourceSuite.scala | 24 ++++++++++++++--------
 .../parquet/ParquetFileFormatSuite.scala           | 12 +++++++----
 .../spark/sql/hive/orc/OrcFileOperator.scala       |  4 ++--
 7 files changed, 42 insertions(+), 38 deletions(-)

diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json
index 8a4afd3133b..522cdd6095a 100644
--- a/core/src/main/resources/error/error-classes.json
+++ b/core/src/main/resources/error/error-classes.json
@@ -99,6 +99,11 @@
     ],
     "sqlState" : "22007"
   },
+  "CANNOT_READ_FILE_FOOTER" : {
+    "message" : [
+      "Could not read footer for file: <file>."
+    ]
+  },
   "CANNOT_RECOGNIZE_HIVE_TYPE" : {
     "message" : [
       "Cannot recognize hive type string: <fieldType>, column: <fieldName>."
@@ -4041,16 +4046,6 @@
       "The input filter of <owner> should be fully convertible."
     ]
   },
-  "_LEGACY_ERROR_TEMP_2091" : {
-    "message" : [
-      "Could not read footer for file: <file>."
-    ]
-  },
-  "_LEGACY_ERROR_TEMP_2092" : {
-    "message" : [
-      "Could not read footer for file: <file>."
-    ]
-  },
   "_LEGACY_ERROR_TEMP_2093" : {
     "message" : [
       "Found duplicate field(s) \"<requiredFieldName>\": <matchedOrcFields> in case-insensitive mode."
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
index 7eca9c3cd10..129d369085b 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
@@ -1065,16 +1065,9 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase {
       cause = null)
   }
 
-  def cannotReadFooterForFileError(file: Path, e: IOException): Throwable = {
+  def cannotReadFooterForFileError(file: Path, e: Exception): Throwable = {
     new SparkException(
-      errorClass = "_LEGACY_ERROR_TEMP_2091",
-      messageParameters = Map("file" -> file.toString()),
-      cause = e)
-  }
-
-  def cannotReadFooterForFileError(file: FileStatus, e: RuntimeException): Throwable = {
-    new SparkException(
-      errorClass = "_LEGACY_ERROR_TEMP_2092",
+      errorClass = "CANNOT_READ_FILE_FOOTER",
       messageParameters = Map("file" -> file.toString()),
       cause = e)
   }
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala
index 5789f252c3b..c7abf62c6d6 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala
@@ -432,7 +432,7 @@ object ParquetFileFormat extends Logging {
           logWarning(s"Skipped the footer in the corrupted file: $currentFile", e)
           None
         } else {
-          throw QueryExecutionErrors.cannotReadFooterForFileError(currentFile, e)
+          throw QueryExecutionErrors.cannotReadFooterForFileError(currentFile.getPath, e)
         }
       }
     }.flatten
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala
index 6757747a81a..f12f882ebe3 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala
@@ -609,10 +609,14 @@ abstract class OrcQueryTest extends OrcTest {
         testIgnoreCorruptFilesWithoutSchemaInfer()
       }
       assert(e2.getMessage.contains("Malformed ORC file"))
-      val e3 = intercept[SparkException] {
-        testAllCorruptFiles()
-      }
-      assert(e3.getMessage.contains("Could not read footer for file"))
+      checkError(
+        exception = intercept[SparkException] {
+          testAllCorruptFiles()
+        },
+        errorClass = "CANNOT_READ_FILE_FOOTER",
+        parameters = Map("file" -> "file:.*"),
+        matchPVals = true
+      )
       val e4 = intercept[SparkException] {
         testAllCorruptFilesWithoutSchemaInfer()
       }
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala
index 94ce3d77962..0fc7cd33bef 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala
@@ -230,10 +230,14 @@ abstract class OrcSuite
   protected def testMergeSchemasInParallel(
       schemaReader: (Seq[FileStatus], Configuration, Boolean) => Seq[StructType]): Unit = {
     testMergeSchemasInParallel(true, schemaReader)
-    val exception = intercept[SparkException] {
-      testMergeSchemasInParallel(false, schemaReader)
-    }.getCause
-    assert(exception.getCause.getMessage.contains("Could not read footer for file"))
+    checkError(
+      exception = intercept[SparkException] {
+        testMergeSchemasInParallel(false, schemaReader)
+      }.getCause.getCause.asInstanceOf[SparkException],
+      errorClass = "CANNOT_READ_FILE_FOOTER",
+      parameters = Map("file" -> "file:.*"),
+      matchPVals = true
+    )
   }
 
   test("create temporary orc table") {
@@ -476,10 +480,14 @@ abstract class OrcSuite
 
         // don't ignore corrupt files
         withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "false") {
-          val exception = intercept[SparkException] {
-            spark.read.orc(basePath).columns.length
-          }.getCause
-          assert(exception.getCause.getMessage.contains("Could not read footer for file"))
+          checkError(
+            exception = intercept[SparkException] {
+              spark.read.orc(basePath).columns.length
+            }.getCause.getCause.asInstanceOf[SparkException],
+            errorClass = "CANNOT_READ_FILE_FOOTER",
+            parameters = Map("file" -> "file:.*"),
+            matchPVals = true
+          )
         }
       }
     }
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormatSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormatSuite.scala
index fa142065265..f3e54e4b75e 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormatSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormatSuite.scala
@@ -61,10 +61,14 @@ abstract class ParquetFileFormatSuite
     }
 
     testReadFooters(true)
-    val exception = intercept[SparkException] {
-      testReadFooters(false)
-    }.getCause
-    assert(exception.getMessage().contains("Could not read footer for file"))
+    checkError(
+      exception = intercept[SparkException] {
+        testReadFooters(false)
+      }.getCause.asInstanceOf[SparkException],
+      errorClass = "CANNOT_READ_FILE_FOOTER",
+      parameters = Map("file" -> "file:.*"),
+      matchPVals = true
+    )
   }
 
   test("SPARK-36825, SPARK-36854: year-month/day-time intervals written and read as INT32/INT64") {
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala
index f6a85c4778b..3fdd5a9c4ce 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala
@@ -24,10 +24,10 @@ import org.apache.hadoop.fs.{FileStatus, Path}
 import org.apache.hadoop.hive.ql.io.orc.{OrcFile, Reader}
 import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector
 
-import org.apache.spark.SparkException
 import org.apache.spark.deploy.SparkHadoopUtil
 import org.apache.spark.internal.Logging
 import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
+import org.apache.spark.sql.errors.QueryExecutionErrors
 import org.apache.spark.sql.types.StructType
 import org.apache.spark.util.ThreadUtils
 
@@ -80,7 +80,7 @@ private[hive] object OrcFileOperator extends Logging {
             logWarning(s"Skipped the footer in the corrupted file: $path", e)
             None
           } else {
-            throw new SparkException(s"Could not read footer for file: $path", e)
+            throw QueryExecutionErrors.cannotReadFooterForFileError(path, e)
           }
       }
       path -> reader


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