This is an automated email from the ASF dual-hosted git repository.
maxgekk 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 3e8cd994a8df [SPARK-47262][SQL] Assign names to error conditions for
parquet conversions
3e8cd994a8df is described below
commit 3e8cd994a8df412a3c788aed63e9cf1ce7ce09e6
Author: Vladan Vasić <[email protected]>
AuthorDate: Thu Aug 22 18:15:33 2024 +0200
[SPARK-47262][SQL] Assign names to error conditions for parquet conversions
### What changes were proposed in this pull request?
In the PR, I propose to rename the legacy error classes
`_LEGACY_ERROR_TEMP_2238`, `_LEGACY_ERROR_TEMP_2239` and
`_LEGACY_ERROR_TEMP_2240` to
`PARQUET_CONVERSION_FAILURE.WITHOUT_DECIMAL_METADATA`,
`PARQUET_CONVERSION_FAILURE.DECIMAL` and `PARQUET_CONVERSION_FAILURE`.
### Why are the changes needed?
Proper name improves user experience w/ Spark SQL.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
No testing was needed.
### Was this patch authored or co-authored using generative AI tooling?
No
Closes #47802 from
vladanvasi-db/vladanvasi-db/parquet-conversion-error-classes-fix.
Authored-by: Vladan Vasić <[email protected]>
Signed-off-by: Max Gekk <[email protected]>
---
.../src/main/resources/error/error-conditions.json | 38 +++++++++++++---------
.../spark/sql/errors/QueryExecutionErrors.scala | 12 +++----
.../parquet/ParquetTypeWideningSuite.scala | 2 +-
3 files changed, 30 insertions(+), 22 deletions(-)
diff --git a/common/utils/src/main/resources/error/error-conditions.json
b/common/utils/src/main/resources/error/error-conditions.json
index 0d29e1e65837..9228b02e3747 100644
--- a/common/utils/src/main/resources/error/error-conditions.json
+++ b/common/utils/src/main/resources/error/error-conditions.json
@@ -3580,6 +3580,29 @@
],
"sqlState" : "42805"
},
+ "PARQUET_CONVERSION_FAILURE" : {
+ "message" : [
+ "Unable to create a Parquet converter for the data type <dataType> whose
Parquet type is <parquetType>."
+ ],
+ "subClass" : {
+ "DECIMAL" : {
+ "message" : [
+ "Parquet DECIMAL type can only be backed by INT32, INT64,
FIXED_LEN_BYTE_ARRAY, or BINARY."
+ ]
+ },
+ "UNSUPPORTED" : {
+ "message" : [
+ "Please modify the conversion making sure it is supported."
+ ]
+ },
+ "WITHOUT_DECIMAL_METADATA" : {
+ "message" : [
+ "Please read this column/field as Spark BINARY type."
+ ]
+ }
+ },
+ "sqlState" : "42846"
+ },
"PARQUET_TYPE_ILLEGAL" : {
"message" : [
"Illegal Parquet type: <parquetType>."
@@ -7321,21 +7344,6 @@
"<className>.getParentLogger is not yet implemented."
]
},
- "_LEGACY_ERROR_TEMP_2238" : {
- "message" : [
- "Unable to create Parquet converter for <typeName> whose Parquet type is
<parquetType> without decimal metadata. Please read this column/field as Spark
BINARY type."
- ]
- },
- "_LEGACY_ERROR_TEMP_2239" : {
- "message" : [
- "Unable to create Parquet converter for decimal type <t> whose Parquet
type is <parquetType>. Parquet DECIMAL type can only be backed by INT32,
INT64, FIXED_LEN_BYTE_ARRAY, or BINARY."
- ]
- },
- "_LEGACY_ERROR_TEMP_2240" : {
- "message" : [
- "Unable to create Parquet converter for data type <t> whose Parquet type
is <parquetType>."
- ]
- },
"_LEGACY_ERROR_TEMP_2241" : {
"message" : [
"Nonatomic partition table <tableName> can not add multiple partitions."
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 3767a4f50857..ac36788c1d0f 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
@@ -1943,27 +1943,27 @@ private[sql] object QueryExecutionErrors extends
QueryErrorsBase with ExecutionE
def cannotCreateParquetConverterForTypeError(
t: DecimalType, parquetType: String): SparkRuntimeException = {
new SparkRuntimeException(
- errorClass = "_LEGACY_ERROR_TEMP_2238",
+ errorClass = "PARQUET_CONVERSION_FAILURE.WITHOUT_DECIMAL_METADATA",
messageParameters = Map(
- "typeName" -> t.typeName,
+ "dataType" -> toSQLType(t),
"parquetType" -> parquetType))
}
def cannotCreateParquetConverterForDecimalTypeError(
t: DecimalType, parquetType: String): SparkRuntimeException = {
new SparkRuntimeException(
- errorClass = "_LEGACY_ERROR_TEMP_2239",
+ errorClass = "PARQUET_CONVERSION_FAILURE.DECIMAL",
messageParameters = Map(
- "t" -> t.json,
+ "dataType" -> toSQLType(t),
"parquetType" -> parquetType))
}
def cannotCreateParquetConverterForDataTypeError(
t: DataType, parquetType: String): SparkRuntimeException = {
new SparkRuntimeException(
- errorClass = "_LEGACY_ERROR_TEMP_2240",
+ errorClass = "PARQUET_CONVERSION_FAILURE.UNSUPPORTED",
messageParameters = Map(
- "t" -> t.json,
+ "dataType" -> toSQLType(t),
"parquetType" -> parquetType))
}
diff --git
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTypeWideningSuite.scala
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTypeWideningSuite.scala
index 4bd35e0789bd..09ed6955a516 100644
---
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTypeWideningSuite.scala
+++
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTypeWideningSuite.scala
@@ -80,7 +80,7 @@ class ParquetTypeWideningSuite
exception.getCause
.isInstanceOf[org.apache.parquet.io.ParquetDecodingException] ||
exception.getCause.getMessage.contains(
- "Unable to create Parquet converter for data type"))
+ "PARQUET_CONVERSION_FAILURE"))
} else {
checkAnswer(readParquetFiles(dir, toType),
expected.select($"a".cast(toType)))
}
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]